View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.mapreduce;
20  
21  import static org.junit.Assert.assertEquals;
22  
23  import com.google.common.base.Joiner;
24  
25  import com.google.common.collect.Sets;
26  import org.apache.commons.cli.CommandLine;
27  import org.apache.commons.lang.RandomStringUtils;
28  import org.apache.commons.logging.Log;
29  import org.apache.commons.logging.LogFactory;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.Path;
32  import org.apache.hadoop.hbase.Cell;
33  import org.apache.hadoop.hbase.CellUtil;
34  import org.apache.hadoop.hbase.HBaseConfiguration;
35  import org.apache.hadoop.hbase.HBaseTestingUtility;
36  import org.apache.hadoop.hbase.HTableDescriptor;
37  import org.apache.hadoop.hbase.IntegrationTestBase;
38  import org.apache.hadoop.hbase.IntegrationTestingUtility;
39  import org.apache.hadoop.hbase.KeyValue;
40  import org.apache.hadoop.hbase.TableName;
41  import org.apache.hadoop.hbase.client.Admin;
42  import org.apache.hadoop.hbase.client.Connection;
43  import org.apache.hadoop.hbase.client.ConnectionFactory;
44  import org.apache.hadoop.hbase.client.Consistency;
45  import org.apache.hadoop.hbase.client.RegionLocator;
46  import org.apache.hadoop.hbase.client.Result;
47  import org.apache.hadoop.hbase.client.Scan;
48  import org.apache.hadoop.hbase.client.Table;
49  import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
50  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
51  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
52  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
53  import org.apache.hadoop.hbase.regionserver.InternalScanner;
54  import org.apache.hadoop.hbase.regionserver.RegionScanner;
55  import org.apache.hadoop.hbase.testclassification.IntegrationTests;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
58  import org.apache.hadoop.hbase.util.RegionSplitter;
59  import org.apache.hadoop.io.LongWritable;
60  import org.apache.hadoop.io.NullWritable;
61  import org.apache.hadoop.io.Writable;
62  import org.apache.hadoop.io.WritableComparable;
63  import org.apache.hadoop.io.WritableComparator;
64  import org.apache.hadoop.io.WritableUtils;
65  import org.apache.hadoop.mapreduce.InputFormat;
66  import org.apache.hadoop.mapreduce.InputSplit;
67  import org.apache.hadoop.mapreduce.Job;
68  import org.apache.hadoop.mapreduce.JobContext;
69  import org.apache.hadoop.mapreduce.Mapper;
70  import org.apache.hadoop.mapreduce.Partitioner;
71  import org.apache.hadoop.mapreduce.RecordReader;
72  import org.apache.hadoop.mapreduce.Reducer;
73  import org.apache.hadoop.mapreduce.TaskAttemptContext;
74  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
75  import org.apache.hadoop.util.StringUtils;
76  import org.apache.hadoop.util.ToolRunner;
77  import org.junit.Test;
78  import org.junit.experimental.categories.Category;
79  
80  import java.io.DataInput;
81  import java.io.DataOutput;
82  import java.io.IOException;
83  import java.util.ArrayList;
84  import java.util.HashSet;
85  import java.util.List;
86  import java.util.Map;
87  import java.util.Random;
88  import java.util.Set;
89  import java.util.concurrent.atomic.AtomicLong;
90  
91  /**
92   * Test Bulk Load and MR on a distributed cluster.
93   * It starts an MR job that creates linked chains
94   *
95   * The format of rows is like this:
96   * Row Key -> Long
97   *
98   * L:<< Chain Id >> -> Row Key of the next link in the chain
99   * S:<< Chain Id >> -> The step in the chain that his link is.
100  * D:<< Chain Id >> -> Random Data.
101  *
102  * All chains start on row 0.
103  * All rk's are > 0.
104  *
105  * After creating the linked lists they are walked over using a TableMapper based Mapreduce Job.
106  *
107  * There are a few options exposed:
108  *
109  * hbase.IntegrationTestBulkLoad.chainLength
110  * The number of rows that will be part of each and every chain.
111  *
112  * hbase.IntegrationTestBulkLoad.numMaps
113  * The number of mappers that will be run.  Each mapper creates on linked list chain.
114  *
115  * hbase.IntegrationTestBulkLoad.numImportRounds
116  * How many jobs will be run to create linked lists.
117  *
118  * hbase.IntegrationTestBulkLoad.tableName
119  * The name of the table.
120  *
121  * hbase.IntegrationTestBulkLoad.replicaCount
122  * How many region replicas to configure for the table under test.
123  */
124 @Category(IntegrationTests.class)
125 public class IntegrationTestBulkLoad extends IntegrationTestBase {
126 
127   private static final Log LOG = LogFactory.getLog(IntegrationTestBulkLoad.class);
128 
129   private static final byte[] CHAIN_FAM = Bytes.toBytes("L");
130   private static final byte[] SORT_FAM  = Bytes.toBytes("S");
131   private static final byte[] DATA_FAM  = Bytes.toBytes("D");
132 
133   private static String CHAIN_LENGTH_KEY = "hbase.IntegrationTestBulkLoad.chainLength";
134   private static int CHAIN_LENGTH = 500000;
135 
136   private static String NUM_MAPS_KEY = "hbase.IntegrationTestBulkLoad.numMaps";
137   private static int NUM_MAPS = 1;
138 
139   private static String NUM_IMPORT_ROUNDS_KEY = "hbase.IntegrationTestBulkLoad.numImportRounds";
140   private static int NUM_IMPORT_ROUNDS = 1;
141 
142   private static String ROUND_NUM_KEY = "hbase.IntegrationTestBulkLoad.roundNum";
143 
144   private static String TABLE_NAME_KEY = "hbase.IntegrationTestBulkLoad.tableName";
145   private static String TABLE_NAME = "IntegrationTestBulkLoad";
146 
147   private static String NUM_REPLICA_COUNT_KEY = "hbase.IntegrationTestBulkLoad.replicaCount";
148   private static int NUM_REPLICA_COUNT_DEFAULT = 1;
149 
150   private static final String OPT_LOAD = "load";
151   private static final String OPT_CHECK = "check";
152 
153   private boolean load = false;
154   private boolean check = false;
155 
156   public static class SlowMeCoproScanOperations extends BaseRegionObserver {
157     static final AtomicLong sleepTime = new AtomicLong(2000);
158     Random r = new Random();
159     AtomicLong countOfNext = new AtomicLong(0);
160     AtomicLong countOfOpen = new AtomicLong(0);
161     public SlowMeCoproScanOperations() {}
162     @Override
163     public RegionScanner preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
164         final Scan scan, final RegionScanner s) throws IOException {
165       if (countOfOpen.incrementAndGet() == 2) { //slowdown openScanner randomly
166         slowdownCode(e);
167       }
168       return s;
169     }
170 
171     @Override
172     public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
173         final InternalScanner s, final List<Result> results,
174         final int limit, final boolean hasMore) throws IOException {
175       //this will slow down a certain next operation if the conditions are met. The slowness
176       //will allow the call to go to a replica
177       countOfNext.incrementAndGet();
178       if (countOfNext.get() == 0 || countOfNext.get() == 4) {
179         slowdownCode(e);
180       }
181       return true;
182     }
183     protected void slowdownCode(final ObserverContext<RegionCoprocessorEnvironment> e) {
184       if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() == 0) {
185         try {
186           if (sleepTime.get() > 0) {
187             LOG.info("Sleeping for " + sleepTime.get() + " ms");
188             Thread.sleep(sleepTime.get());
189           }
190         } catch (InterruptedException e1) {
191           LOG.error(e1);
192         }
193       }
194     }
195   }
196 
197   /**
198    * Modify table {@code getTableName()} to carry {@link SlowMeCoproScanOperations}.
199    */
200   private void installSlowingCoproc() throws IOException, InterruptedException {
201     int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
202     if (replicaCount == NUM_REPLICA_COUNT_DEFAULT) return;
203 
204     TableName t = getTablename();
205     Admin admin = util.getHBaseAdmin();
206     HTableDescriptor desc = admin.getTableDescriptor(t);
207     desc.addCoprocessor(SlowMeCoproScanOperations.class.getName());
208     HBaseTestingUtility.modifyTableSync(admin, desc);
209   }
210 
211   @Test
212   public void testBulkLoad() throws Exception {
213     runLoad();
214     installSlowingCoproc();
215     runCheckWithRetry();
216   }
217 
218   public void runLoad() throws Exception {
219     setupTable();
220     int numImportRounds = getConf().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
221     LOG.info("Running load with numIterations:" + numImportRounds);
222     for (int i = 0; i < numImportRounds; i++) {
223       runLinkedListMRJob(i);
224     }
225   }
226 
227   private byte[][] getSplits(int numRegions) {
228     RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
229     split.setFirstRow(Bytes.toBytes(0L));
230     split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
231     return split.split(numRegions);
232   }
233 
234   private void setupTable() throws IOException, InterruptedException {
235     if (util.getHBaseAdmin().tableExists(getTablename())) {
236       util.deleteTable(getTablename());
237     }
238 
239     util.createTable(
240         getTablename().getName(),
241         new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
242         getSplits(16)
243     );
244 
245     int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
246     if (replicaCount == NUM_REPLICA_COUNT_DEFAULT) return;
247 
248     TableName t = getTablename();
249     HBaseTestingUtility.setReplicas(util.getHBaseAdmin(), t, replicaCount);
250   }
251 
252   private void runLinkedListMRJob(int iteration) throws Exception {
253     String jobName =  IntegrationTestBulkLoad.class.getSimpleName() + " - " +
254         EnvironmentEdgeManager.currentTime();
255     Configuration conf = new Configuration(util.getConfiguration());
256     Path p = null;
257     if (conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY) == null) {
258       p = util.getDataTestDirOnTestFS(getTablename() + "-" + iteration);
259     } else {
260       p = new Path(conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY));
261     }
262 
263     conf.setBoolean("mapreduce.map.speculative", false);
264     conf.setBoolean("mapreduce.reduce.speculative", false);
265     conf.setInt(ROUND_NUM_KEY, iteration);
266 
267     Job job = new Job(conf);
268 
269     job.setJobName(jobName);
270 
271     // set the input format so that we can create map tasks with no data input.
272     job.setInputFormatClass(ITBulkLoadInputFormat.class);
273 
274     // Set the mapper classes.
275     job.setMapperClass(LinkedListCreationMapper.class);
276     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
277     job.setMapOutputValueClass(KeyValue.class);
278 
279     // Use the identity reducer
280     // So nothing to do here.
281 
282     // Set this jar.
283     job.setJarByClass(getClass());
284 
285     // Set where to place the hfiles.
286     FileOutputFormat.setOutputPath(job, p);
287     try (Connection conn = ConnectionFactory.createConnection(conf);
288         Admin admin = conn.getAdmin();
289         Table table = conn.getTable(getTablename());
290         RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
291 
292       // Configure the partitioner and other things needed for HFileOutputFormat.
293       HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
294 
295       // Run the job making sure it works.
296       assertEquals(true, job.waitForCompletion(true));
297 
298       // Create a new loader.
299       LoadIncrementalHFiles loader = new LoadIncrementalHFiles(conf);
300 
301       // Load the HFiles in.
302       loader.doBulkLoad(p, admin, table, regionLocator);
303     }
304 
305     // Delete the files.
306     util.getTestFileSystem().delete(p, true);
307   }
308 
309   public static class EmptySplit extends InputSplit implements Writable {
310     @Override
311     public void write(DataOutput out) throws IOException { }
312     @Override
313     public void readFields(DataInput in) throws IOException { }
314     @Override
315     public long getLength() { return 0L; }
316     @Override
317     public String[] getLocations() { return new String[0]; }
318   }
319 
320   public static class FixedRecordReader<K, V> extends RecordReader<K, V> {
321     private int index = -1;
322     private K[] keys;
323     private V[] values;
324 
325     public FixedRecordReader(K[] keys, V[] values) {
326       this.keys = keys;
327       this.values = values;
328     }
329     @Override
330     public void initialize(InputSplit split, TaskAttemptContext context) throws IOException,
331     InterruptedException { }
332     @Override
333     public boolean nextKeyValue() throws IOException, InterruptedException {
334       return ++index < keys.length;
335     }
336     @Override
337     public K getCurrentKey() throws IOException, InterruptedException {
338       return keys[index];
339     }
340     @Override
341     public V getCurrentValue() throws IOException, InterruptedException {
342       return values[index];
343     }
344     @Override
345     public float getProgress() throws IOException, InterruptedException {
346       return (float)index / keys.length;
347     }
348     @Override
349     public void close() throws IOException {
350     }
351   }
352 
353   public static class ITBulkLoadInputFormat extends InputFormat<LongWritable, LongWritable> {
354     @Override
355     public List<InputSplit> getSplits(JobContext context) throws IOException, InterruptedException {
356       int numSplits = context.getConfiguration().getInt(NUM_MAPS_KEY, NUM_MAPS);
357       ArrayList<InputSplit> ret = new ArrayList<InputSplit>(numSplits);
358       for (int i = 0; i < numSplits; ++i) {
359         ret.add(new EmptySplit());
360       }
361       return ret;
362     }
363 
364     @Override
365     public RecordReader<LongWritable, LongWritable> createRecordReader(InputSplit split,
366       TaskAttemptContext context)
367           throws IOException, InterruptedException {
368       int taskId = context.getTaskAttemptID().getTaskID().getId();
369       int numMapTasks = context.getConfiguration().getInt(NUM_MAPS_KEY, NUM_MAPS);
370       int numIterations = context.getConfiguration().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
371       int iteration = context.getConfiguration().getInt(ROUND_NUM_KEY, 0);
372 
373       taskId = taskId + iteration * numMapTasks;
374       numMapTasks = numMapTasks * numIterations;
375 
376       long chainId = Math.abs(new Random().nextLong());
377       chainId = chainId - (chainId % numMapTasks) + taskId; // ensure that chainId is unique per task and across iterations
378       LongWritable[] keys = new LongWritable[] {new LongWritable(chainId)};
379 
380       return new FixedRecordReader<LongWritable, LongWritable>(keys, keys);
381     }
382   }
383 
384   /**
385    * Mapper that creates a linked list of KeyValues.
386    *
387    * Each map task generates one linked list.
388    * All lists start on row key 0L.
389    * All lists should be CHAIN_LENGTH long.
390    */
391   public static class LinkedListCreationMapper
392       extends Mapper<LongWritable, LongWritable, ImmutableBytesWritable, KeyValue> {
393 
394     private Random rand = new Random();
395 
396     @Override
397     protected void map(LongWritable key, LongWritable value, Context context)
398         throws IOException, InterruptedException {
399       long chainId = value.get();
400       LOG.info("Starting mapper with chainId:" + chainId);
401 
402       byte[] chainIdArray = Bytes.toBytes(chainId);
403       long currentRow = 0;
404 
405       long chainLength = context.getConfiguration().getLong(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
406       long nextRow = getNextRow(0, chainLength);
407 
408       for (long i = 0; i < chainLength; i++) {
409         byte[] rk = Bytes.toBytes(currentRow);
410 
411         // Next link in the chain.
412         KeyValue linkKv = new KeyValue(rk, CHAIN_FAM, chainIdArray, Bytes.toBytes(nextRow));
413         // What link in the chain this is.
414         KeyValue sortKv = new KeyValue(rk, SORT_FAM, chainIdArray, Bytes.toBytes(i));
415         // Added data so that large stores are created.
416         KeyValue dataKv = new KeyValue(rk, DATA_FAM, chainIdArray,
417           Bytes.toBytes(RandomStringUtils.randomAlphabetic(50))
418         );
419 
420         // Emit the key values.
421         context.write(new ImmutableBytesWritable(rk), linkKv);
422         context.write(new ImmutableBytesWritable(rk), sortKv);
423         context.write(new ImmutableBytesWritable(rk), dataKv);
424         // Move to the next row.
425         currentRow = nextRow;
426         nextRow = getNextRow(i+1, chainLength);
427       }
428     }
429 
430     /** Returns a unique row id within this chain for this index */
431     private long getNextRow(long index, long chainLength) {
432       long nextRow = Math.abs(rand.nextLong());
433       // use significant bits from the random number, but pad with index to ensure it is unique
434       // this also ensures that we do not reuse row = 0
435       // row collisions from multiple mappers are fine, since we guarantee unique chainIds
436       nextRow = nextRow - (nextRow % chainLength) + index;
437       return nextRow;
438     }
439   }
440 
441   /**
442    * Writable class used as the key to group links in the linked list.
443    *
444    * Used as the key emited from a pass over the table.
445    */
446   public static class LinkKey implements WritableComparable<LinkKey> {
447 
448     private Long chainId;
449 
450     public Long getOrder() {
451       return order;
452     }
453 
454     public Long getChainId() {
455       return chainId;
456     }
457 
458     private Long order;
459 
460     public LinkKey() {}
461 
462     public LinkKey(long chainId, long order) {
463       this.chainId = chainId;
464       this.order = order;
465     }
466 
467     @Override
468     public int compareTo(LinkKey linkKey) {
469       int res = getChainId().compareTo(linkKey.getChainId());
470       if (res == 0) {
471         res = getOrder().compareTo(linkKey.getOrder());
472       }
473       return res;
474     }
475 
476     @Override
477     public void write(DataOutput dataOutput) throws IOException {
478       WritableUtils.writeVLong(dataOutput, chainId);
479       WritableUtils.writeVLong(dataOutput, order);
480     }
481 
482     @Override
483     public void readFields(DataInput dataInput) throws IOException {
484       chainId = WritableUtils.readVLong(dataInput);
485       order = WritableUtils.readVLong(dataInput);
486     }
487   }
488 
489   /**
490    * Writable used as the value emitted from a pass over the hbase table.
491    */
492   public static class LinkChain implements WritableComparable<LinkChain> {
493 
494     public Long getNext() {
495       return next;
496     }
497 
498     public Long getRk() {
499       return rk;
500     }
501 
502     public LinkChain() {}
503 
504     public LinkChain(Long rk, Long next) {
505       this.rk = rk;
506       this.next = next;
507     }
508 
509     private Long rk;
510     private Long next;
511 
512     @Override
513     public int compareTo(LinkChain linkChain) {
514       int res = getRk().compareTo(linkChain.getRk());
515       if (res == 0) {
516         res = getNext().compareTo(linkChain.getNext());
517       }
518       return res;
519     }
520 
521     @Override
522     public void write(DataOutput dataOutput) throws IOException {
523       WritableUtils.writeVLong(dataOutput, rk);
524       WritableUtils.writeVLong(dataOutput, next);
525     }
526 
527     @Override
528     public void readFields(DataInput dataInput) throws IOException {
529       rk = WritableUtils.readVLong(dataInput);
530       next = WritableUtils.readVLong(dataInput);
531     }
532   }
533 
534   /**
535    * Class to figure out what partition to send a link in the chain to.  This is based upon
536    * the linkKey's ChainId.
537    */
538   public static class NaturalKeyPartitioner extends Partitioner<LinkKey, LinkChain> {
539     @Override
540     public int getPartition(LinkKey linkKey,
541                             LinkChain linkChain,
542                             int numPartitions) {
543       int hash = linkKey.getChainId().hashCode();
544       return Math.abs(hash % numPartitions);
545     }
546   }
547 
548   /**
549    * Comparator used to figure out if a linkKey should be grouped together.  This is based upon the
550    * linkKey's ChainId.
551    */
552   public static class NaturalKeyGroupingComparator extends WritableComparator {
553 
554     protected NaturalKeyGroupingComparator() {
555       super(LinkKey.class, true);
556     }
557 
558     @Override
559     public int compare(WritableComparable w1, WritableComparable w2) {
560       LinkKey k1 = (LinkKey) w1;
561       LinkKey k2 = (LinkKey) w2;
562 
563       return k1.getChainId().compareTo(k2.getChainId());
564     }
565   }
566 
567   /**
568    * Comparator used to order linkKeys so that they are passed to a reducer in order.  This is based
569    * upon linkKey ChainId and Order.
570    */
571   public static class CompositeKeyComparator extends WritableComparator {
572 
573     protected CompositeKeyComparator() {
574       super(LinkKey.class, true);
575     }
576 
577     @Override
578     public int compare(WritableComparable w1, WritableComparable w2) {
579       LinkKey k1 = (LinkKey) w1;
580       LinkKey k2 = (LinkKey) w2;
581 
582       return k1.compareTo(k2);
583     }
584   }
585 
586   /**
587    * Mapper to pass over the table.
588    *
589    * For every row there could be multiple chains that landed on this row. So emit a linkKey
590    * and value for each.
591    */
592   public static class LinkedListCheckingMapper extends TableMapper<LinkKey, LinkChain> {
593     @Override
594     protected void map(ImmutableBytesWritable key, Result value, Context context)
595         throws IOException, InterruptedException {
596       long longRk = Bytes.toLong(value.getRow());
597 
598       for (Map.Entry<byte[], byte[]> entry : value.getFamilyMap(CHAIN_FAM).entrySet()) {
599         long chainId = Bytes.toLong(entry.getKey());
600         long next = Bytes.toLong(entry.getValue());
601         Cell c = value.getColumnCells(SORT_FAM, entry.getKey()).get(0);
602         long order = Bytes.toLong(CellUtil.cloneValue(c));
603         context.write(new LinkKey(chainId, order), new LinkChain(longRk, next));
604       }
605     }
606   }
607 
608   /**
609    * Class that does the actual checking of the links.
610    *
611    * All links in the chain should be grouped and sorted when sent to this class.  Then the chain
612    * will be traversed making sure that no link is missing and that the chain is the correct length.
613    *
614    * This will throw an exception if anything is not correct.  That causes the job to fail if any
615    * data is corrupt.
616    */
617   public static class LinkedListCheckingReducer
618       extends Reducer<LinkKey, LinkChain, NullWritable, NullWritable> {
619     @Override
620     protected void reduce(LinkKey key, Iterable<LinkChain> values, Context context)
621         throws java.io.IOException, java.lang.InterruptedException {
622       long next = -1L;
623       long prev = -1L;
624       long count = 0L;
625 
626       for (LinkChain lc : values) {
627 
628         if (next == -1) {
629           if (lc.getRk() != 0L) {
630             String msg = "Chains should all start at rk 0, but read rk " + lc.getRk()
631                 + ". Chain:" + key.chainId + ", order:" + key.order;
632             logError(msg, context);
633             throw new RuntimeException(msg);
634           }
635           next = lc.getNext();
636         } else {
637           if (next != lc.getRk()) {
638             String msg = "Missing a link in the chain. Prev rk " + prev + " was, expecting "
639                 + next + " but got " + lc.getRk() + ". Chain:" + key.chainId
640                 + ", order:" + key.order;
641             logError(msg, context);
642             throw new RuntimeException(msg);
643           }
644           prev = lc.getRk();
645           next = lc.getNext();
646         }
647         count++;
648       }
649 
650       int expectedChainLen = context.getConfiguration().getInt(CHAIN_LENGTH_KEY, CHAIN_LENGTH);
651       if (count != expectedChainLen) {
652         String msg = "Chain wasn't the correct length.  Expected " + expectedChainLen + " got "
653             + count + ". Chain:" + key.chainId + ", order:" + key.order;
654         logError(msg, context);
655         throw new RuntimeException(msg);
656       }
657     }
658 
659     private static void logError(String msg, Context context) throws IOException {
660       TableName table = getTableName(context.getConfiguration());
661 
662       LOG.error("Failure in chain verification: " + msg);
663       try (Connection connection = ConnectionFactory.createConnection(context.getConfiguration());
664           Admin admin = connection.getAdmin()) {
665         LOG.error("cluster status:\n" + admin.getClusterStatus());
666         LOG.error("table regions:\n"
667             + Joiner.on("\n").join(admin.getTableRegions(table)));
668       }
669     }
670   }
671 
672   private void runCheckWithRetry() throws IOException, ClassNotFoundException, InterruptedException {
673     try {
674       runCheck();
675     } catch (Throwable t) {
676       LOG.warn("Received " + StringUtils.stringifyException(t));
677       LOG.warn("Running the check MR Job again to see whether an ephemeral problem or not");
678       runCheck();
679       throw t; // we should still fail the test even if second retry succeeds
680     }
681     // everything green
682   }
683 
684 
685   /**
686    * After adding data to the table start a mr job to
687    * @throws IOException
688    * @throws ClassNotFoundException
689    * @throws InterruptedException
690    */
691   private void runCheck() throws IOException, ClassNotFoundException, InterruptedException {
692     LOG.info("Running check");
693     Configuration conf = getConf();
694     String jobName = getTablename() + "_check" + EnvironmentEdgeManager.currentTime();
695     Path p = util.getDataTestDirOnTestFS(jobName);
696 
697     Job job = new Job(conf);
698     job.setJarByClass(getClass());
699     job.setJobName(jobName);
700 
701     job.setPartitionerClass(NaturalKeyPartitioner.class);
702     job.setGroupingComparatorClass(NaturalKeyGroupingComparator.class);
703     job.setSortComparatorClass(CompositeKeyComparator.class);
704 
705     Scan scan = new Scan();
706     scan.addFamily(CHAIN_FAM);
707     scan.addFamily(SORT_FAM);
708     scan.setMaxVersions(1);
709     scan.setCacheBlocks(false);
710     scan.setBatch(1000);
711 
712     int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
713     if (replicaCount != NUM_REPLICA_COUNT_DEFAULT) {
714       scan.setConsistency(Consistency.TIMELINE);
715     }
716 
717     TableMapReduceUtil.initTableMapperJob(
718         getTablename().getName(),
719         scan,
720         LinkedListCheckingMapper.class,
721         LinkKey.class,
722         LinkChain.class,
723         job
724     );
725 
726     job.setReducerClass(LinkedListCheckingReducer.class);
727     job.setOutputKeyClass(NullWritable.class);
728     job.setOutputValueClass(NullWritable.class);
729 
730     FileOutputFormat.setOutputPath(job, p);
731 
732     assertEquals(true, job.waitForCompletion(true));
733 
734     // Delete the files.
735     util.getTestFileSystem().delete(p, true);
736   }
737 
738   @Override
739   public void setUpCluster() throws Exception {
740     util = getTestingUtil(getConf());
741     util.initializeCluster(1);
742     int replicaCount = getConf().getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
743     if (LOG.isDebugEnabled() && replicaCount != NUM_REPLICA_COUNT_DEFAULT) {
744       LOG.debug("Region Replicas enabled: " + replicaCount);
745     }
746 
747     // Scale this up on a real cluster
748     if (util.isDistributedCluster()) {
749       util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
750           Integer.toString(util.getHBaseAdmin().getClusterStatus().getServersSize() * 10)
751       );
752       util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
753     } else {
754       util.startMiniMapReduceCluster();
755     }
756   }
757 
758   @Override
759   protected void addOptions() {
760     super.addOptions();
761     super.addOptNoArg(OPT_CHECK, "Run check only");
762     super.addOptNoArg(OPT_LOAD, "Run load only");
763   }
764 
765   @Override
766   protected void processOptions(CommandLine cmd) {
767     super.processOptions(cmd);
768     check = cmd.hasOption(OPT_CHECK);
769     load = cmd.hasOption(OPT_LOAD);
770   }
771 
772   @Override
773   public int runTestFromCommandLine() throws Exception {
774     if (load) {
775       runLoad();
776     } else if (check) {
777       installSlowingCoproc();
778       runCheckWithRetry();
779     } else {
780       testBulkLoad();
781     }
782     return 0;
783   }
784 
785   @Override
786   public TableName getTablename() {
787     return getTableName(getConf());
788   }
789 
790   public static TableName getTableName(Configuration conf) {
791     return TableName.valueOf(conf.get(TABLE_NAME_KEY, TABLE_NAME));
792   }
793 
794   @Override
795   protected Set<String> getColumnFamilies() {
796     return Sets.newHashSet(Bytes.toString(CHAIN_FAM) , Bytes.toString(DATA_FAM),
797         Bytes.toString(SORT_FAM));
798   }
799 
800   public static void main(String[] args) throws Exception {
801     Configuration conf = HBaseConfiguration.create();
802     IntegrationTestingUtility.setUseDistributedCluster(conf);
803     int status =  ToolRunner.run(conf, new IntegrationTestBulkLoad(), args);
804     System.exit(status);
805   }
806 }