001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.mapreduce;
020
021import static org.junit.Assert.assertEquals;
022
023import java.io.DataInput;
024import java.io.DataOutput;
025import java.io.IOException;
026import java.util.ArrayList;
027import java.util.List;
028import java.util.Map;
029import java.util.Optional;
030import java.util.Random;
031import java.util.Set;
032import java.util.concurrent.atomic.AtomicLong;
033import org.apache.commons.lang3.RandomStringUtils;
034import org.apache.hadoop.conf.Configuration;
035import org.apache.hadoop.fs.Path;
036import org.apache.hadoop.hbase.Cell;
037import org.apache.hadoop.hbase.CellUtil;
038import org.apache.hadoop.hbase.HBaseConfiguration;
039import org.apache.hadoop.hbase.HBaseTestingUtil;
040import org.apache.hadoop.hbase.IntegrationTestBase;
041import org.apache.hadoop.hbase.IntegrationTestingUtility;
042import org.apache.hadoop.hbase.KeyValue;
043import org.apache.hadoop.hbase.TableName;
044import org.apache.hadoop.hbase.client.Admin;
045import org.apache.hadoop.hbase.client.Connection;
046import org.apache.hadoop.hbase.client.ConnectionFactory;
047import org.apache.hadoop.hbase.client.Consistency;
048import org.apache.hadoop.hbase.client.RegionLocator;
049import org.apache.hadoop.hbase.client.Result;
050import org.apache.hadoop.hbase.client.Scan;
051import org.apache.hadoop.hbase.client.TableDescriptor;
052import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
053import org.apache.hadoop.hbase.coprocessor.ObserverContext;
054import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor;
055import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
056import org.apache.hadoop.hbase.coprocessor.RegionObserver;
057import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
058import org.apache.hadoop.hbase.regionserver.InternalScanner;
059import org.apache.hadoop.hbase.testclassification.IntegrationTests;
060import org.apache.hadoop.hbase.tool.BulkLoadHFiles;
061import org.apache.hadoop.hbase.util.Bytes;
062import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
063import org.apache.hadoop.hbase.util.RegionSplitter;
064import org.apache.hadoop.io.LongWritable;
065import org.apache.hadoop.io.NullWritable;
066import org.apache.hadoop.io.Writable;
067import org.apache.hadoop.io.WritableComparable;
068import org.apache.hadoop.io.WritableComparator;
069import org.apache.hadoop.io.WritableUtils;
070import org.apache.hadoop.mapreduce.InputFormat;
071import org.apache.hadoop.mapreduce.InputSplit;
072import org.apache.hadoop.mapreduce.Job;
073import org.apache.hadoop.mapreduce.JobContext;
074import org.apache.hadoop.mapreduce.Mapper;
075import org.apache.hadoop.mapreduce.Partitioner;
076import org.apache.hadoop.mapreduce.RecordReader;
077import org.apache.hadoop.mapreduce.Reducer;
078import org.apache.hadoop.mapreduce.TaskAttemptContext;
079import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
080import org.apache.hadoop.util.StringUtils;
081import org.apache.hadoop.util.ToolRunner;
082import org.junit.Test;
083import org.junit.experimental.categories.Category;
084import org.slf4j.Logger;
085import org.slf4j.LoggerFactory;
086
087import org.apache.hbase.thirdparty.com.google.common.base.Joiner;
088import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
089import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
090
091/**
092 * Test Bulk Load and MR on a distributed cluster.
093 * It starts an MR job that creates linked chains
094 *
095 * The format of rows is like this:
096 * Row Key -> Long
097 *
098 * L:<< Chain Id >> -> Row Key of the next link in the chain
099 * 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)
125public class IntegrationTestBulkLoad extends IntegrationTestBase {
126
127  private static final Logger LOG = LoggerFactory.getLogger(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 implements RegionCoprocessor, RegionObserver {
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
163    @Override
164    public Optional<RegionObserver> getRegionObserver() {
165      return Optional.of(this);
166    }
167
168    @Override
169    public void preScannerOpen(final ObserverContext<RegionCoprocessorEnvironment> e,
170        final Scan scan) throws IOException {
171      if (countOfOpen.incrementAndGet() == 2) { //slowdown openScanner randomly
172        slowdownCode(e);
173      }
174    }
175
176    @Override
177    public boolean preScannerNext(final ObserverContext<RegionCoprocessorEnvironment> e,
178        final InternalScanner s, final List<Result> results,
179        final int limit, final boolean hasMore) throws IOException {
180      //this will slow down a certain next operation if the conditions are met. The slowness
181      //will allow the call to go to a replica
182      countOfNext.incrementAndGet();
183      if (countOfNext.get() == 0 || countOfNext.get() == 4) {
184        slowdownCode(e);
185      }
186      return true;
187    }
188    protected void slowdownCode(final ObserverContext<RegionCoprocessorEnvironment> e) {
189      if (e.getEnvironment().getRegion().getRegionInfo().getReplicaId() == 0) {
190        try {
191          if (sleepTime.get() > 0) {
192            LOG.info("Sleeping for " + sleepTime.get() + " ms");
193            Thread.sleep(sleepTime.get());
194          }
195        } catch (InterruptedException e1) {
196          LOG.error(e1.toString(), e1);
197        }
198      }
199    }
200  }
201
202  /**
203   * Modify table {@code getTableName()} to carry {@link SlowMeCoproScanOperations}.
204   */
205  private void installSlowingCoproc() throws IOException, InterruptedException {
206    int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
207    if (replicaCount == NUM_REPLICA_COUNT_DEFAULT) return;
208
209    TableName t = getTablename();
210    Admin admin = util.getAdmin();
211    TableDescriptor desc = admin.getDescriptor(t);
212    TableDescriptorBuilder builder = TableDescriptorBuilder.newBuilder(desc);
213    builder.setCoprocessor(SlowMeCoproScanOperations.class.getName());
214    admin.modifyTable(builder.build());
215  }
216
217  @Test
218  public void testBulkLoad() throws Exception {
219    runLoad();
220    installSlowingCoproc();
221    runCheckWithRetry();
222  }
223
224  public void runLoad() throws Exception {
225    setupTable();
226    int numImportRounds = getConf().getInt(NUM_IMPORT_ROUNDS_KEY, NUM_IMPORT_ROUNDS);
227    LOG.info("Running load with numIterations:" + numImportRounds);
228    for (int i = 0; i < numImportRounds; i++) {
229      runLinkedListMRJob(i);
230    }
231  }
232
233  private byte[][] getSplits(int numRegions) {
234    RegionSplitter.UniformSplit split = new RegionSplitter.UniformSplit();
235    split.setFirstRow(Bytes.toBytes(0L));
236    split.setLastRow(Bytes.toBytes(Long.MAX_VALUE));
237    return split.split(numRegions);
238  }
239
240  private void setupTable() throws IOException, InterruptedException {
241    if (util.getAdmin().tableExists(getTablename())) {
242      util.deleteTable(getTablename());
243    }
244
245    util.createTable(
246        getTablename(),
247        new byte[][]{CHAIN_FAM, SORT_FAM, DATA_FAM},
248        getSplits(16)
249    );
250
251    int replicaCount = conf.getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
252    if (replicaCount == NUM_REPLICA_COUNT_DEFAULT) return;
253
254    TableName t = getTablename();
255    HBaseTestingUtil.setReplicas(util.getAdmin(), t, replicaCount);
256  }
257
258  private void runLinkedListMRJob(int iteration) throws Exception {
259    String jobName =  IntegrationTestBulkLoad.class.getSimpleName() + " - " +
260        EnvironmentEdgeManager.currentTime();
261    Configuration conf = new Configuration(util.getConfiguration());
262    Path p = null;
263    if (conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY) == null) {
264      p = util.getDataTestDirOnTestFS(getTablename() + "-" + iteration);
265    } else {
266      p = new Path(conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY));
267    }
268
269    conf.setBoolean("mapreduce.map.speculative", false);
270    conf.setBoolean("mapreduce.reduce.speculative", false);
271    conf.setInt(ROUND_NUM_KEY, iteration);
272
273    Job job = new Job(conf);
274
275    job.setJobName(jobName);
276
277    // set the input format so that we can create map tasks with no data input.
278    job.setInputFormatClass(ITBulkLoadInputFormat.class);
279
280    // Set the mapper classes.
281    job.setMapperClass(LinkedListCreationMapper.class);
282    job.setMapOutputKeyClass(ImmutableBytesWritable.class);
283    job.setMapOutputValueClass(KeyValue.class);
284
285    // Use the identity reducer
286    // So nothing to do here.
287
288    // Set this jar.
289    job.setJarByClass(getClass());
290
291    // Set where to place the hfiles.
292    FileOutputFormat.setOutputPath(job, p);
293    try (Connection conn = ConnectionFactory.createConnection(conf); Admin admin = conn.getAdmin();
294      RegionLocator regionLocator = conn.getRegionLocator(getTablename())) {
295      // Configure the partitioner and other things needed for HFileOutputFormat.
296      HFileOutputFormat2.configureIncrementalLoad(job, admin.getDescriptor(getTablename()),
297        regionLocator);
298      // Run the job making sure it works.
299      assertEquals(true, job.waitForCompletion(true));
300    }
301    // Create a new loader.
302    BulkLoadHFiles loader = BulkLoadHFiles.create(conf);
303    // Load the HFiles in.
304    loader.bulkLoad(getTablename(), p);
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<>(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<>(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 metrics:\n" + admin.getClusterMetrics());
666        LOG.error("table regions:\n"
667            + Joiner.on("\n").join(admin.getRegions(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.readVersions(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.getAdmin().getRegionServers().size() * 10));
751      util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
752    } else {
753      util.startMiniMapReduceCluster();
754    }
755  }
756
757  @Override
758  protected void addOptions() {
759    super.addOptions();
760    super.addOptNoArg(OPT_CHECK, "Run check only");
761    super.addOptNoArg(OPT_LOAD, "Run load only");
762  }
763
764  @Override
765  protected void processOptions(CommandLine cmd) {
766    super.processOptions(cmd);
767    check = cmd.hasOption(OPT_CHECK);
768    load = cmd.hasOption(OPT_LOAD);
769  }
770
771  @Override
772  public int runTestFromCommandLine() throws Exception {
773    if (load) {
774      runLoad();
775    } else if (check) {
776      installSlowingCoproc();
777      runCheckWithRetry();
778    } else {
779      testBulkLoad();
780    }
781    return 0;
782  }
783
784  @Override
785  public TableName getTablename() {
786    return getTableName(getConf());
787  }
788
789  public static TableName getTableName(Configuration conf) {
790    return TableName.valueOf(conf.get(TABLE_NAME_KEY, TABLE_NAME));
791  }
792
793  @Override
794  protected Set<String> getColumnFamilies() {
795    return Sets.newHashSet(Bytes.toString(CHAIN_FAM) , Bytes.toString(DATA_FAM),
796        Bytes.toString(SORT_FAM));
797  }
798
799  public static void main(String[] args) throws Exception {
800    Configuration conf = HBaseConfiguration.create();
801    IntegrationTestingUtility.setUseDistributedCluster(conf);
802    int status =  ToolRunner.run(conf, new IntegrationTestBulkLoad(), args);
803    System.exit(status);
804  }
805}