1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce;
20
21 import static org.junit.Assert.assertEquals;
22
23 import java.io.DataInput;
24 import java.io.DataOutput;
25 import java.io.IOException;
26 import java.util.ArrayList;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.Random;
30 import java.util.Set;
31
32 import com.google.common.base.Joiner;
33 import org.apache.commons.cli.CommandLine;
34 import org.apache.commons.lang.RandomStringUtils;
35 import org.apache.commons.logging.Log;
36 import org.apache.commons.logging.LogFactory;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.Path;
39 import org.apache.hadoop.hbase.*;
40 import org.apache.hadoop.hbase.client.HTable;
41 import org.apache.hadoop.hbase.client.Result;
42 import org.apache.hadoop.hbase.client.Scan;
43 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
44 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
45 import org.apache.hadoop.hbase.util.Bytes;
46 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
47 import org.apache.hadoop.hbase.util.RegionSplitter;
48 import org.apache.hadoop.io.LongWritable;
49 import org.apache.hadoop.io.NullWritable;
50 import org.apache.hadoop.io.Writable;
51 import org.apache.hadoop.io.WritableComparable;
52 import org.apache.hadoop.io.WritableComparator;
53 import org.apache.hadoop.io.WritableUtils;
54 import org.apache.hadoop.mapreduce.InputFormat;
55 import org.apache.hadoop.mapreduce.InputSplit;
56 import org.apache.hadoop.mapreduce.Job;
57 import org.apache.hadoop.mapreduce.JobContext;
58 import org.apache.hadoop.mapreduce.Mapper;
59 import org.apache.hadoop.mapreduce.Partitioner;
60 import org.apache.hadoop.mapreduce.RecordReader;
61 import org.apache.hadoop.mapreduce.Reducer;
62 import org.apache.hadoop.mapreduce.TaskAttemptContext;
63 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
64 import org.apache.hadoop.util.ToolRunner;
65 import org.junit.Test;
66 import org.junit.experimental.categories.Category;
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99 @Category(IntegrationTests.class)
100 public class IntegrationTestBulkLoad extends IntegrationTestBase {
101
102 private static final Log LOG = LogFactory.getLog(IntegrationTestBulkLoad.class);
103
104 private static byte[] CHAIN_FAM = Bytes.toBytes("L");
105 private static byte[] SORT_FAM = Bytes.toBytes("S");
106 private static byte[] DATA_FAM = Bytes.toBytes("D");
107
108 private static String CHAIN_LENGTH_KEY = "hbase.IntegrationTestBulkLoad.chainLength";
109 private static int CHAIN_LENGTH = 500000;
110
111 private static String NUM_MAPS_KEY = "hbase.IntegrationTestBulkLoad.numMaps";
112 private static int NUM_MAPS = 1;
113
114 private static String NUM_IMPORT_ROUNDS_KEY = "hbase.IntegrationTestBulkLoad.numImportRounds";
115 private static int NUM_IMPORT_ROUNDS = 1;
116
117 private static String ROUND_NUM_KEY = "hbase.IntegrationTestBulkLoad.roundNum";
118
119 private static String TABLE_NAME_KEY = "hbase.IntegrationTestBulkLoad.tableName";
120 private static String TABLE_NAME = "IntegrationTestBulkLoad";
121
122 @Test
123 public void testBulkLoad() throws Exception {
124 runLoad();
125 runCheck();
126 }
127
128 public void runLoad() throws Exception {
129 setupTable();
130 int numImportRounds = getConf().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
131 LOG.info("Running load with numIterations:" + numImportRounds);
132 for (int i = 0; i < numImportRounds; i++) {
133 runLinkedListMRJob(i);
134 }
135 }
136
137 private byte[][] getSplits(int numRegions) {
138 RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
139 split.setFirstRow(Bytes.toBytes(0L));
140 split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
141 return split.split(numRegions);
142 }
143
144 private void setupTable() throws IOException {
145 if (util.getHBaseAdmin().tableExists(getTablename())) {
146 util.deleteTable(getTablename());
147 }
148
149 util.createTable(
150 Bytes.toBytes(getTablename()),
151 new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
152 getSplits(16)
153 );
154 }
155
156 private void runLinkedListMRJob(int iteration) throws Exception {
157 String jobName = IntegrationTestBulkLoad.class.getSimpleName() + " - " +
158 EnvironmentEdgeManager.currentTimeMillis();
159 Configuration conf = new Configuration(util.getConfiguration());
160 Path p = null;
161 if (conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY) == null) {
162 p = util.getDataTestDirOnTestFS(getTablename() + "-" + iteration);
163 } else {
164 p = new Path(conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY));
165 }
166 HTable table = new HTable(conf, getTablename());
167
168 conf.setBoolean("mapreduce.map.speculative", false);
169 conf.setBoolean("mapreduce.reduce.speculative", false);
170 conf.setInt(ROUND_NUM_KEY, iteration);
171
172 Job job = new Job(conf);
173
174 job.setJobName(jobName);
175
176
177 job.setInputFormatClass(ITBulkLoadInputFormat.class);
178
179
180 job.setMapperClass(LinkedListCreationMapper.class);
181 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
182 job.setMapOutputValueClass(KeyValue.class);
183
184
185
186
187
188 job.setJarByClass(getClass());
189
190
191 FileOutputFormat.setOutputPath(job, p);
192
193
194 HFileOutputFormat.configureIncrementalLoad(job, table);
195
196
197 assertEquals(true, job.waitForCompletion(true));
198
199
200 LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
201
202
203 loader.doBulkLoad(p, table);
204
205
206 util.getTestFileSystem().delete(p, true);
207 }
208
209 public static class EmptySplit extends InputSplit implements Writable {
210 @Override
211 public void write(DataOutput out) throws IOException { }
212 @Override
213 public void readFields(DataInput in) throws IOException { }
214 @Override
215 public long getLength() { return 0L; }
216 @Override
217 public String[] getLocations() { return new String[0]; }
218 }
219
220 public static class FixedRecordReader<K, V> extends RecordReader<K, V> {
221 private int index = -1;
222 private K[] keys;
223 private V[] values;
224
225 public FixedRecordReader(K[] keys, V[] values) {
226 this.keys = keys;
227 this.values = values;
228 }
229 @Override
230 public void initialize(InputSplit split, TaskAttemptContext context) throws IOException,
231 InterruptedException { }
232 @Override
233 public boolean nextKeyValue() throws IOException, InterruptedException {
234 return ++index < keys.length;
235 }
236 @Override
237 public K getCurrentKey() throws IOException, InterruptedException {
238 return keys[index];
239 }
240 @Override
241 public V getCurrentValue() throws IOException, InterruptedException {
242 return values[index];
243 }
244 @Override
245 public float getProgress() throws IOException, InterruptedException {
246 return (float)index / keys.length;
247 }
248 @Override
249 public void close() throws IOException {
250 }
251 }
252
253 public static class ITBulkLoadInputFormat extends InputFormat<LongWritable, LongWritable> {
254 @Override
255 public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
256 int numSplits = context.getConfiguration().getInt(NUM_MAPS_KEY, NUM_MAPS);
257 ArrayList<InputSplit> ret = new ArrayList<InputSplit>(numSplits);
258 for (int i = 0; i < numSplits; ++i) {
259 ret.add(new EmptySplit());
260 }
261 return ret;
262 }
263
264 @Override
265 public RecordReader<LongWritable, LongWritable> createRecordReader(InputSplit split,
266 TaskAttemptContext context)
267 throws IOException, InterruptedException {
268 int taskId = context.getTaskAttemptID().getTaskID().getId();
269 int numMapTasks = context.getConfiguration().getInt(NUM_MAPS_KEY, NUM_MAPS);
270 int numIterations = context.getConfiguration().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
271 int iteration = context.getConfiguration().getInt(ROUND_NUM_KEY, 0);
272
273 taskId = taskId + iteration * numMapTasks;
274 numMapTasks = numMapTasks * numIterations;
275
276 long chainId = Math.abs(new Random().nextLong());
277 chainId = chainId - (chainId % numMapTasks) + taskId;
278 LongWritable[] keys = new LongWritable[] {new LongWritable(chainId)};
279
280 return new FixedRecordReader<LongWritable, LongWritable>(keys, keys);
281 }
282 }
283
284
285
286
287
288
289
290
291 public static class LinkedListCreationMapper
292 extends Mapper<LongWritable, LongWritable, ImmutableBytesWritable, KeyValue> {
293
294 private Random rand = new Random();
295
296 @Override
297 protected void map(LongWritable key, LongWritable value, Context context)
298 throws IOException, InterruptedException {
299 long chainId = value.get();
300 LOG.info("Starting mapper with chainId:" + chainId);
301
302 byte[] chainIdArray = Bytes.toBytes(chainId);
303 long currentRow = 0;
304
305 long chainLength = context.getConfiguration().getLong(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
306 long nextRow = getNextRow(0, chainLength);
307
308 for (long i = 0; i < chainLength; i++) {
309 byte[] rk = Bytes.toBytes(currentRow);
310
311
312 KeyValue linkKv = new KeyValue(rk, CHAIN_FAM, chainIdArray, Bytes.toBytes(nextRow));
313
314 KeyValue sortKv = new KeyValue(rk, SORT_FAM, chainIdArray, Bytes.toBytes(i));
315
316 KeyValue dataKv = new KeyValue(rk, DATA_FAM, chainIdArray,
317 Bytes.toBytes(RandomStringUtils.randomAlphabetic(50))
318 );
319
320
321 context.write(new ImmutableBytesWritable(rk), linkKv);
322 context.write(new ImmutableBytesWritable(rk), sortKv);
323 context.write(new ImmutableBytesWritable(rk), dataKv);
324
325 currentRow = nextRow;
326 nextRow = getNextRow(i+1, chainLength);
327 }
328 }
329
330
331 private long getNextRow(long index, long chainLength) {
332 long nextRow = Math.abs(rand.nextLong());
333
334
335
336 nextRow = nextRow - (nextRow % chainLength) + index;
337 return nextRow;
338 }
339 }
340
341
342
343
344
345
346 public static class LinkKey implements WritableComparable<LinkKey> {
347
348 private Long chainId;
349
350 public Long getOrder() {
351 return order;
352 }
353
354 public Long getChainId() {
355 return chainId;
356 }
357
358 private Long order;
359
360 public LinkKey() {}
361
362 public LinkKey(long chainId, long order) {
363 this.chainId = chainId;
364 this.order = order;
365 }
366
367 @Override
368 public int compareTo(LinkKey linkKey) {
369 int res = getChainId().compareTo(linkKey.getChainId());
370 if (res == 0) {
371 res = getOrder().compareTo(linkKey.getOrder());
372 }
373 return res;
374 }
375
376 @Override
377 public void write(DataOutput dataOutput) throws IOException {
378 WritableUtils.writeVLong(dataOutput, chainId);
379 WritableUtils.writeVLong(dataOutput, order);
380 }
381
382 @Override
383 public void readFields(DataInput dataInput) throws IOException {
384 chainId = WritableUtils.readVLong(dataInput);
385 order = WritableUtils.readVLong(dataInput);
386 }
387 }
388
389
390
391
392 public static class LinkChain implements WritableComparable<LinkChain> {
393
394 public Long getNext() {
395 return next;
396 }
397
398 public Long getRk() {
399 return rk;
400 }
401
402 public LinkChain() {}
403
404 public LinkChain(Long rk, Long next) {
405 this.rk = rk;
406 this.next = next;
407 }
408
409 private Long rk;
410 private Long next;
411
412 @Override
413 public int compareTo(LinkChain linkChain) {
414 int res = getRk().compareTo(linkChain.getRk());
415 if (res == 0) {
416 res = getNext().compareTo(linkChain.getNext());
417 }
418 return res;
419 }
420
421 @Override
422 public void write(DataOutput dataOutput) throws IOException {
423 WritableUtils.writeVLong(dataOutput, rk);
424 WritableUtils.writeVLong(dataOutput, next);
425 }
426
427 @Override
428 public void readFields(DataInput dataInput) throws IOException {
429 rk = WritableUtils.readVLong(dataInput);
430 next = WritableUtils.readVLong(dataInput);
431 }
432 }
433
434
435
436
437
438 public static class NaturalKeyPartitioner extends Partitioner<LinkKey, LinkChain> {
439 @Override
440 public int getPartition(LinkKey linkKey,
441 LinkChain linkChain,
442 int numPartitions) {
443 int hash = linkKey.getChainId().hashCode();
444 return Math.abs(hash % numPartitions);
445 }
446 }
447
448
449
450
451
452 public static class NaturalKeyGroupingComparator extends WritableComparator {
453
454 protected NaturalKeyGroupingComparator() {
455 super(LinkKey.class, true);
456 }
457
458 @Override
459 public int compare(WritableComparable w1, WritableComparable w2) {
460 LinkKey k1 = (LinkKey) w1;
461 LinkKey k2 = (LinkKey) w2;
462
463 return k1.getChainId().compareTo(k2.getChainId());
464 }
465 }
466
467
468
469
470
471 public static class CompositeKeyComparator extends WritableComparator {
472
473 protected CompositeKeyComparator() {
474 super(LinkKey.class, true);
475 }
476
477 @Override
478 public int compare(WritableComparable w1, WritableComparable w2) {
479 LinkKey k1 = (LinkKey) w1;
480 LinkKey k2 = (LinkKey) w2;
481
482 return k1.compareTo(k2);
483 }
484 }
485
486
487
488
489
490
491
492 public static class LinkedListCheckingMapper extends TableMapper<LinkKey, LinkChain> {
493 @Override
494 protected void map(ImmutableBytesWritable key, Result value, Context context)
495 throws IOException, InterruptedException {
496 long longRk = Bytes.toLong(value.getRow());
497
498 for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
499 long chainId = Bytes.toLong(entry.getKey());
500 long next = Bytes.toLong(entry.getValue());
501 Cell c = value.getColumnCells(SORT_FAM, entry.getKey()).get(0);
502 long order = Bytes.toLong(CellUtil.cloneValue(c));
503 context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
504 }
505 }
506 }
507
508
509
510
511
512
513
514
515
516
517 public static class LinkedListCheckingReducer
518 extends Reducer<LinkKey, LinkChain, NullWritable, NullWritable> {
519 @Override
520 protected void reduce(LinkKey key, Iterable<LinkChain> values, Context context)
521 throws java.io.IOException, java.lang.InterruptedException {
522 long next = -1L;
523 long prev = -1L;
524 long count = 0L;
525
526 for (LinkChain lc : values) {
527
528 if (next == -1) {
529 if (lc.getRk() != 0L) {
530 String msg = "Chains should all start at rk 0, but read rk " + lc.getRk()
531 + ". Chain:" + key.chainId + ", order:" + key.order;
532 logError(msg, context);
533 throw new RuntimeException(msg);
534 }
535 next = lc.getNext();
536 } else {
537 if (next != lc.getRk()) {
538 String msg = "Missing a link in the chain. Prev rk " + prev + " was, expecting "
539 + next + " but got " + lc.getRk() + ". Chain:" + key.chainId
540 + ", order:" + key.order;
541 logError(msg, context);
542 throw new RuntimeException(msg);
543 }
544 prev = lc.getRk();
545 next = lc.getNext();
546 }
547 count++;
548 }
549
550 int expectedChainLen = context.getConfiguration().getInt(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
551 if (count != expectedChainLen) {
552 String msg = "Chain wasn't the correct length. Expected " + expectedChainLen + " got "
553 + count + ". Chain:" + key.chainId + ", order:" + key.order;
554 logError(msg, context);
555 throw new RuntimeException(msg);
556 }
557 }
558
559 private static void logError(String msg, Context context) throws IOException {
560 HBaseTestingUtility util = new HBaseTestingUtility(context.getConfiguration());
561 TableName table = TableName.valueOf(getTableName(context.getConfiguration()));
562
563 LOG.error("Failure in chain verification: " + msg);
564 LOG.error("cluster status:\n" + util.getHBaseClusterInterface().getClusterStatus());
565 LOG.error("table regions:\n"
566 + Joiner.on("\n").join(util.getHBaseAdmin().getTableRegions(table)));
567 }
568 }
569
570
571
572
573
574
575
576 private void runCheck() throws IOException, ClassNotFoundException, InterruptedException {
577 LOG.info("Running check");
578 Configuration conf = getConf();
579 String jobName = getTablename() + "_check" + EnvironmentEdgeManager.currentTimeMillis();
580 Path p = util.getDataTestDirOnTestFS(jobName);
581
582 Job job = new Job(conf);
583
584 job.setJarByClass(getClass());
585
586 job.setPartitionerClass(NaturalKeyPartitioner.class);
587 job.setGroupingComparatorClass(NaturalKeyGroupingComparator.class);
588 job.setSortComparatorClass(CompositeKeyComparator.class);
589
590 Scan s = new Scan();
591 s.addFamily(CHAIN_FAM);
592 s.addFamily(SORT_FAM);
593 s.setMaxVersions(1);
594 s.setCacheBlocks(false);
595 s.setBatch(1000);
596
597 TableMapReduceUtil.initTableMapperJob(
598 Bytes.toBytes(getTablename()),
599 new Scan(),
600 LinkedListCheckingMapper.class,
601 LinkKey.class,
602 LinkChain.class,
603 job
604 );
605
606 job.setReducerClass(LinkedListCheckingReducer.class);
607 job.setOutputKeyClass(NullWritable.class);
608 job.setOutputValueClass(NullWritable.class);
609
610 FileOutputFormat.setOutputPath(job, p);
611
612 assertEquals(true, job.waitForCompletion(true));
613
614
615 util.getTestFileSystem().delete(p, true);
616 }
617
618 @Override
619 public void setUpCluster() throws Exception {
620 util = getTestingUtil(getConf());
621 util.initializeCluster(1);
622
623
624 if (util.isDistributedCluster()) {
625 util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
626 Integer.toString(util.getHBaseAdmin().getClusterStatus().getServersSize() * 10)
627 );
628 util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
629 } else {
630 util.startMiniMapReduceCluster();
631 }
632 }
633
634 private static final String OPT_LOAD = "load";
635 private static final String OPT_CHECK = "check";
636
637 private boolean load = false;
638 private boolean check = false;
639
640 @Override
641 protected void addOptions() {
642 super.addOptions();
643 super.addOptNoArg(OPT_CHECK, "Run check only");
644 super.addOptNoArg(OPT_LOAD, "Run load only");
645 }
646
647 @Override
648 protected void processOptions(CommandLine cmd) {
649 super.processOptions(cmd);
650 check = cmd.hasOption(OPT_CHECK);
651 load = cmd.hasOption(OPT_LOAD);
652 }
653
654 @Override
655 public int runTestFromCommandLine() throws Exception {
656 if (load) {
657 runLoad();
658 } else if (check) {
659 runCheck();
660 } else {
661 testBulkLoad();
662 }
663 return 0;
664 }
665
666 @Override
667 public String getTablename() {
668 return getTableName(getConf());
669 }
670
671 public static String getTableName(Configuration conf) {
672 return conf.get(TABLE_NAME_KEY, TABLE_NAME);
673 }
674
675 @Override
676 protected Set<String> getColumnFamilies() {
677 return null;
678 }
679
680 public static void main(String[] args) throws Exception {
681 Configuration conf = HBaseConfiguration.create();
682 IntegrationTestingUtility.setUseDistributedCluster(conf);
683 int status = ToolRunner.run(conf, new IntegrationTestBulkLoad(), args);
684 System.exit(status);
685 }
686
687 }