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