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