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 java.io.DataInput;
22 import java.io.DataOutput;
23 import java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Random;
28 import java.util.Set;
29
30 import org.apache.commons.lang.RandomStringUtils;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hbase.HBaseConfiguration;
34 import org.apache.hadoop.hbase.IntegrationTestBase;
35 import org.apache.hadoop.hbase.IntegrationTestingUtility;
36 import org.apache.hadoop.hbase.IntegrationTests;
37 import org.apache.hadoop.hbase.KeyValue;
38 import org.apache.hadoop.hbase.client.HTable;
39 import org.apache.hadoop.hbase.client.Result;
40 import org.apache.hadoop.hbase.client.Scan;
41 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
42 import org.apache.hadoop.hbase.util.Bytes;
43 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
44 import org.apache.hadoop.hbase.util.RegionSplitter;
45 import org.apache.hadoop.io.LongWritable;
46 import org.apache.hadoop.io.NullWritable;
47 import org.apache.hadoop.io.Text;
48 import org.apache.hadoop.io.WritableComparable;
49 import org.apache.hadoop.io.WritableComparator;
50 import org.apache.hadoop.io.WritableUtils;
51 import org.apache.hadoop.mapreduce.InputFormat;
52 import org.apache.hadoop.mapreduce.InputSplit;
53 import org.apache.hadoop.mapreduce.Job;
54 import org.apache.hadoop.mapreduce.JobContext;
55 import org.apache.hadoop.mapreduce.Mapper;
56 import org.apache.hadoop.mapreduce.Partitioner;
57 import org.apache.hadoop.mapreduce.RecordReader;
58 import org.apache.hadoop.mapreduce.Reducer;
59 import org.apache.hadoop.mapreduce.TaskAttemptContext;
60 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
61 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
62 import org.apache.hadoop.util.ToolRunner;
63 import org.junit.After;
64 import org.junit.Before;
65 import org.junit.Test;
66 import org.junit.experimental.categories.Category;
67
68 import static org.junit.Assert.assertEquals;
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
100
101 @Category(IntegrationTests.class)
102 public class IntegrationTestBulkLoad extends IntegrationTestBase {
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
118 private static String TABLE_NAME_KEY = "hbase.IntegrationTestBulkLoad.tableName";
119 private static String TABLE_NAME = "IntegrationTestBulkLoad";
120
121 @Test
122 public void testBulkLoad() throws Exception {
123 setupTable();
124 int numImportRounds = getConf().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
125 for (int i = 0; i < numImportRounds; i++) {
126 runLinkedListMRJob(i);
127 }
128 runCheck();
129 }
130
131 private byte[][] getSplits(int numRegions) {
132 RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
133 split.setFirstRow(Bytes.toBytes(0L));
134 split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
135 return split.split(numRegions);
136 }
137
138 private void setupTable() throws IOException {
139 if (util.getHBaseAdmin().tableExists(getTablename())) {
140 util.deleteTable(getTablename());
141 }
142
143 util.createTable(
144 Bytes.toBytes(getTablename()),
145 new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
146 getSplits(16)
147 );
148 }
149
150 private void runLinkedListMRJob(int iteration) throws Exception {
151 String jobName = IntegrationTestBulkLoad.class.getSimpleName() + " - " +
152 EnvironmentEdgeManager.currentTimeMillis();
153 Configuration conf = new Configuration(util.getConfiguration());
154 Path p = util.getDataTestDirOnTestFS(getTablename() + "-" + iteration);
155 HTable table = new HTable(conf, getTablename());
156
157 conf.setBoolean("mapreduce.map.speculative", false);
158 conf.setBoolean("mapreduce.reduce.speculative", false);
159
160 Job job = new Job(conf);
161
162 job.setJobName(jobName);
163
164
165 job.setInputFormatClass(RandomInputFormat.class);
166
167
168 job.setMapperClass(LinkedListCreationMapper.class);
169 job.setMapOutputKeyClass(ImmutableBytesWritable.class);
170 job.setMapOutputValueClass(KeyValue.class);
171
172
173
174
175
176 job.setJarByClass(getClass());
177
178
179 FileOutputFormat.setOutputPath(job, p);
180
181
182 HFileOutputFormat.configureIncrementalLoad(job, table);
183
184
185 assertEquals(true, job.waitForCompletion(true));
186
187
188 LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
189
190
191 loader.doBulkLoad(p, table);
192
193
194 util.getTestFileSystem().delete(p, true);
195 }
196
197
198
199
200
201
202
203 static class RandomInputFormat extends InputFormat<Text, LongWritable> {
204 public List<InputSplit> getSplits(JobContext job) throws IOException {
205 List<InputSplit> result = new ArrayList<InputSplit>();
206 int numSplits = job.getConfiguration().getInt(NUM_MAPS_KEY, NUM_MAPS);
207 for (int i = 0; i < numSplits; ++i) {
208 result.add(new FileSplit(new Path("/tmp", "dummy-split-" + i), 0, 1, null));
209 }
210 return result;
211 }
212
213
214
215
216
217
218
219 static class RandomRecordReader extends RecordReader<Text, LongWritable> {
220 Path name;
221 Text key = null;
222 LongWritable value = new LongWritable();
223
224 public RandomRecordReader(Path p) {
225 name = p;
226 }
227
228 public void initialize(InputSplit split,
229 TaskAttemptContext context)
230 throws IOException, InterruptedException {
231
232 }
233
234 public boolean nextKeyValue() {
235 if (name != null) {
236 key = new Text();
237 key.set(name.getName());
238 name = null;
239 value.set(new Random().nextLong());
240 return true;
241 }
242 return false;
243 }
244
245 public Text getCurrentKey() {
246 return key;
247 }
248
249 public LongWritable getCurrentValue() {
250 return value;
251 }
252
253 public void close() {
254 }
255
256 public float getProgress() {
257 return 0.0f;
258 }
259 }
260
261 public RecordReader<Text, LongWritable> createRecordReader(InputSplit split,
262 TaskAttemptContext context)
263 throws IOException, InterruptedException {
264 return new RandomRecordReader(((FileSplit) split).getPath());
265 }
266 }
267
268
269
270
271
272
273
274
275 public static class LinkedListCreationMapper
276 extends Mapper<Text, LongWritable, ImmutableBytesWritable, KeyValue> {
277
278 private Random rand = new Random();
279
280 protected void map(Text key, LongWritable value, Context context)
281 throws IOException, InterruptedException {
282
283 long chainId = value.get();
284 byte[] chainIdArray = Bytes.toBytes(chainId);
285 long currentRow = 0;
286 long nextRow = Math.abs(rand.nextLong());
287
288 int chainLength = context.getConfiguration().getInt(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
289
290 for (long i = 0; i < chainLength; i++) {
291 byte[] rk = Bytes.toBytes(currentRow);
292
293
294 KeyValue linkKv = new KeyValue(rk, CHAIN_FAM, chainIdArray, Bytes.toBytes(nextRow));
295
296 KeyValue sortKv = new KeyValue(rk, SORT_FAM, chainIdArray, Bytes.toBytes(i));
297
298 KeyValue dataKv = new KeyValue(rk, DATA_FAM, chainIdArray,
299 Bytes.toBytes(RandomStringUtils.randomAlphabetic(50))
300 );
301
302
303 context.write(new ImmutableBytesWritable(rk), linkKv);
304 context.write(new ImmutableBytesWritable(rk), sortKv);
305 context.write(new ImmutableBytesWritable(rk), dataKv);
306
307 currentRow = nextRow;
308 nextRow = Math.abs(rand.nextLong());
309 }
310 }
311 }
312
313
314
315
316
317
318 public static class LinkKey implements WritableComparable<LinkKey> {
319
320 private Long chainId;
321
322 public Long getOrder() {
323 return order;
324 }
325
326 public Long getChainId() {
327 return chainId;
328 }
329
330 private Long order;
331
332 public LinkKey() {
333
334 }
335
336 public LinkKey(long chainId, long order) {
337 this.chainId = chainId;
338 this.order = order;
339 }
340
341 @Override
342 public int compareTo(LinkKey linkKey) {
343 int res = getChainId().compareTo(linkKey.getChainId());
344 if (res == 0) {
345 res = getOrder().compareTo(linkKey.getOrder());
346 }
347 return res;
348 }
349
350 @Override
351 public void write(DataOutput dataOutput) throws IOException {
352 WritableUtils.writeVLong(dataOutput, chainId);
353 WritableUtils.writeVLong(dataOutput, order);
354 }
355
356 @Override
357 public void readFields(DataInput dataInput) throws IOException {
358 chainId = WritableUtils.readVLong(dataInput);
359 order = WritableUtils.readVLong(dataInput);
360 }
361 }
362
363
364
365
366 public static class LinkChain implements WritableComparable<LinkChain> {
367
368 public Long getNext() {
369 return next;
370 }
371
372 public Long getRk() {
373 return rk;
374 }
375
376 public LinkChain() {
377 }
378
379 public LinkChain(Long rk, Long next) {
380 this.rk = rk;
381 this.next = next;
382 }
383
384 private Long rk;
385 private Long next;
386
387 @Override
388 public int compareTo(LinkChain linkChain) {
389 int res = getRk().compareTo(linkChain.getRk());
390 if (res == 0) {
391 res = getNext().compareTo(linkChain.getNext());
392 }
393 return res;
394 }
395
396 @Override
397 public void write(DataOutput dataOutput) throws IOException {
398 WritableUtils.writeVLong(dataOutput, rk);
399 WritableUtils.writeVLong(dataOutput, next);
400 }
401
402 @Override
403 public void readFields(DataInput dataInput) throws IOException {
404 rk = WritableUtils.readVLong(dataInput);
405 next = WritableUtils.readVLong(dataInput);
406 }
407 }
408
409
410
411
412
413 public static class NaturalKeyPartitioner extends Partitioner<LinkKey, LinkChain> {
414 @Override
415 public int getPartition(LinkKey linkKey,
416 LinkChain linkChain,
417 int numPartitions) {
418 int hash = linkKey.getChainId().hashCode();
419 int partition = hash % numPartitions;
420 return partition;
421 }
422 }
423
424
425
426
427
428 public static class NaturalKeyGroupingComparator extends WritableComparator {
429
430 protected NaturalKeyGroupingComparator() {
431 super(LinkKey.class, true);
432 }
433
434 public int compare(WritableComparable w1, WritableComparable w2) {
435 LinkKey k1 = (LinkKey) w1;
436 LinkKey k2 = (LinkKey) w2;
437
438 return k1.getChainId().compareTo(k2.getChainId());
439 }
440 }
441
442
443
444
445
446 public static class CompositeKeyComparator extends WritableComparator {
447
448 protected CompositeKeyComparator() {
449 super(LinkKey.class, true);
450 }
451
452 @Override
453 public int compare(WritableComparable w1, WritableComparable w2) {
454 LinkKey k1 = (LinkKey) w1;
455 LinkKey k2 = (LinkKey) w2;
456
457 return k1.compareTo(k2);
458 }
459 }
460
461
462
463
464
465
466
467 public static class LinkedListCheckingMapper extends TableMapper<LinkKey, LinkChain> {
468 protected void map(ImmutableBytesWritable key, Result value, Context context)
469 throws IOException, InterruptedException {
470 long longRk = Bytes.toLong(value.getRow());
471
472 for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
473 long chainId = Bytes.toLong(entry.getKey());
474 long next = Bytes.toLong(entry.getValue());
475 long order = Bytes.toLong(value.getColumn(SORT_FAM, entry.getKey()).get(0).getValue());
476 context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
477 }
478 }
479 }
480
481
482
483
484
485
486
487
488
489
490 public static class LinkedListCheckingReducer
491 extends Reducer<LinkKey, LinkChain, NullWritable, NullWritable> {
492 protected void reduce(LinkKey key, Iterable<LinkChain> values, Context context)
493 throws java.io.IOException, java.lang.InterruptedException {
494 long next = -1L;
495 long count = 0L;
496
497 for (LinkChain lc : values) {
498
499 if (next == -1) {
500 if (lc.getRk() != 0L) throw new RuntimeException("Chains should all start at 0 rk");
501 next = lc.getNext();
502 } else {
503 if (next != lc.getRk())
504 throw new RuntimeException("Missing a link in the chain. Expecthing " +
505 next + " got " + lc.getRk());
506 next = lc.getNext();
507 }
508 count++;
509 }
510
511 int expectedChainLen = context.getConfiguration().getInt(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
512 if (count != expectedChainLen)
513 throw new RuntimeException("Chain wasn't the correct length. Expected " +
514 expectedChainLen + " got " + count);
515 }
516 }
517
518
519
520
521
522
523
524 private void runCheck() throws IOException, ClassNotFoundException, InterruptedException {
525 Configuration conf = getConf();
526 String jobName = getTablename() + "_check" + EnvironmentEdgeManager.currentTimeMillis();
527 Path p = util.getDataTestDirOnTestFS(jobName);
528
529 Job job = new Job(conf);
530
531 job.setJarByClass(getClass());
532
533 job.setPartitionerClass(NaturalKeyPartitioner.class);
534 job.setGroupingComparatorClass(NaturalKeyGroupingComparator.class);
535 job.setSortComparatorClass(CompositeKeyComparator.class);
536
537 Scan s = new Scan();
538 s.addFamily(CHAIN_FAM);
539 s.addFamily(SORT_FAM);
540 s.setMaxVersions(1);
541 s.setCacheBlocks(false);
542 s.setBatch(100);
543
544 TableMapReduceUtil.initTableMapperJob(
545 Bytes.toBytes(getTablename()),
546 new Scan(),
547 LinkedListCheckingMapper.class,
548 LinkKey.class,
549 LinkChain.class,
550 job
551 );
552
553 job.setReducerClass(LinkedListCheckingReducer.class);
554 job.setOutputKeyClass(NullWritable.class);
555 job.setOutputValueClass(NullWritable.class);
556
557 FileOutputFormat.setOutputPath(job, p);
558
559 assertEquals(true, job.waitForCompletion(true));
560
561
562 util.getTestFileSystem().delete(p, true);
563 }
564
565 @Before
566 @Override
567 public void setUp() throws Exception {
568 util = getTestingUtil(getConf());
569 util.initializeCluster(1);
570
571
572 if (util.isDistributedCluster()) {
573 util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
574 Integer.toString(util.getHBaseAdmin().getClusterStatus().getServersSize() * 10)
575 );
576 util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
577 } else {
578 util.startMiniMapReduceCluster();
579 }
580 }
581
582 @After
583 @Override
584 public void cleanUp() throws Exception {
585 util.restoreCluster();
586 util = null;
587 }
588
589 @Override
590 public int runTestFromCommandLine() throws Exception {
591 runCheck();
592 return 0;
593 }
594
595 @Override
596 public String getTablename() {
597 return getConf().get(TABLE_NAME_KEY, TABLE_NAME);
598 }
599
600 @Override
601 protected Set<String> getColumnFamilies() {
602 return null;
603 }
604
605 public static void main(String[] args) throws Exception {
606 Configuration conf = HBaseConfiguration.create();
607 IntegrationTestingUtility.setUseDistributedCluster(conf);
608 int status = ToolRunner.run(conf, new IntegrationTestBulkLoad(), args);
609 System.exit(status);
610 }
611
612 }