001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018
019package org.apache.hadoop.hbase.test;
020
021import java.io.DataInput;
022import java.io.DataOutput;
023import java.io.FileNotFoundException;
024import java.io.IOException;
025import java.io.InterruptedIOException;
026import java.security.SecureRandom;
027import java.util.ArrayList;
028import java.util.Arrays;
029import java.util.EnumSet;
030import java.util.Iterator;
031import java.util.List;
032import java.util.Random;
033import java.util.Set;
034import java.util.SortedSet;
035import java.util.TreeSet;
036import java.util.UUID;
037import java.util.concurrent.ThreadLocalRandom;
038import java.util.concurrent.atomic.AtomicInteger;
039import org.apache.hadoop.conf.Configuration;
040import org.apache.hadoop.conf.Configured;
041import org.apache.hadoop.fs.FileSystem;
042import org.apache.hadoop.fs.LocatedFileStatus;
043import org.apache.hadoop.fs.Path;
044import org.apache.hadoop.fs.RemoteIterator;
045import org.apache.hadoop.hbase.Cell;
046import org.apache.hadoop.hbase.ClusterMetrics.Option;
047import org.apache.hadoop.hbase.HBaseConfiguration;
048import org.apache.hadoop.hbase.HBaseTestingUtility;
049import org.apache.hadoop.hbase.HColumnDescriptor;
050import org.apache.hadoop.hbase.HConstants;
051import org.apache.hadoop.hbase.HRegionLocation;
052import org.apache.hadoop.hbase.HTableDescriptor;
053import org.apache.hadoop.hbase.IntegrationTestBase;
054import org.apache.hadoop.hbase.IntegrationTestingUtility;
055import org.apache.hadoop.hbase.MasterNotRunningException;
056import org.apache.hadoop.hbase.TableName;
057import org.apache.hadoop.hbase.client.Admin;
058import org.apache.hadoop.hbase.client.BufferedMutator;
059import org.apache.hadoop.hbase.client.BufferedMutatorParams;
060import org.apache.hadoop.hbase.client.Connection;
061import org.apache.hadoop.hbase.client.ConnectionFactory;
062import org.apache.hadoop.hbase.client.Get;
063import org.apache.hadoop.hbase.client.Mutation;
064import org.apache.hadoop.hbase.client.Put;
065import org.apache.hadoop.hbase.client.RegionLocator;
066import org.apache.hadoop.hbase.client.Result;
067import org.apache.hadoop.hbase.client.ResultScanner;
068import org.apache.hadoop.hbase.client.Scan;
069import org.apache.hadoop.hbase.client.ScannerCallable;
070import org.apache.hadoop.hbase.client.Table;
071import org.apache.hadoop.hbase.fs.HFileSystem;
072import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
073import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
074import org.apache.hadoop.hbase.mapreduce.TableMapper;
075import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
076import org.apache.hadoop.hbase.mapreduce.WALPlayer;
077import org.apache.hadoop.hbase.regionserver.FlushAllLargeStoresPolicy;
078import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
079import org.apache.hadoop.hbase.testclassification.IntegrationTests;
080import org.apache.hadoop.hbase.util.AbstractHBaseTool;
081import org.apache.hadoop.hbase.util.Bytes;
082import org.apache.hadoop.hbase.util.RegionSplitter;
083import org.apache.hadoop.hbase.wal.WALEdit;
084import org.apache.hadoop.hbase.wal.WALKey;
085import org.apache.hadoop.io.BytesWritable;
086import org.apache.hadoop.io.NullWritable;
087import org.apache.hadoop.io.Writable;
088import org.apache.hadoop.mapreduce.Counter;
089import org.apache.hadoop.mapreduce.CounterGroup;
090import org.apache.hadoop.mapreduce.Counters;
091import org.apache.hadoop.mapreduce.InputFormat;
092import org.apache.hadoop.mapreduce.InputSplit;
093import org.apache.hadoop.mapreduce.Job;
094import org.apache.hadoop.mapreduce.JobContext;
095import org.apache.hadoop.mapreduce.Mapper;
096import org.apache.hadoop.mapreduce.RecordReader;
097import org.apache.hadoop.mapreduce.Reducer;
098import org.apache.hadoop.mapreduce.TaskAttemptContext;
099import org.apache.hadoop.mapreduce.TaskAttemptID;
100import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
101import org.apache.hadoop.mapreduce.lib.input.FileSplit;
102import org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat;
103import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
104import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
105import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
106import org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat;
107import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
108import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
109import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
110import org.apache.hadoop.util.Tool;
111import org.apache.hadoop.util.ToolRunner;
112import org.junit.Test;
113import org.junit.experimental.categories.Category;
114import org.slf4j.Logger;
115import org.slf4j.LoggerFactory;
116import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
117import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
118import org.apache.hbase.thirdparty.org.apache.commons.cli.GnuParser;
119import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter;
120import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
121import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
122
123/**
124 * This is an integration test borrowed from goraci, written by Keith Turner,
125 * which is in turn inspired by the Accumulo test called continous ingest (ci).
126 * The original source code can be found here:
127 * https://github.com/keith-turner/goraci
128 * https://github.com/enis/goraci/
129 *
130 * Apache Accumulo [0] has a simple test suite that verifies that data is not
131 * lost at scale. This test suite is called continuous ingest. This test runs
132 * many ingest clients that continually create linked lists containing 25
133 * million nodes. At some point the clients are stopped and a map reduce job is
134 * run to ensure no linked list has a hole. A hole indicates data was lost.··
135 *
136 * The nodes in the linked list are random. This causes each linked list to
137 * spread across the table. Therefore if one part of a table loses data, then it
138 * will be detected by references in another part of the table.
139 *
140 * THE ANATOMY OF THE TEST
141 *
142 * Below is rough sketch of how data is written. For specific details look at
143 * the Generator code.
144 *
145 * 1 Write out 1 million nodes· 2 Flush the client· 3 Write out 1 million that
146 * reference previous million· 4 If this is the 25th set of 1 million nodes,
147 * then update 1st set of million to point to last· 5 goto 1
148 *
149 * The key is that nodes only reference flushed nodes. Therefore a node should
150 * never reference a missing node, even if the ingest client is killed at any
151 * point in time.
152 *
153 * When running this test suite w/ Accumulo there is a script running in
154 * parallel called the Aggitator that randomly and continuously kills server
155 * processes.·· The outcome was that many data loss bugs were found in Accumulo
156 * by doing this.· This test suite can also help find bugs that impact uptime
157 * and stability when· run for days or weeks.··
158 *
159 * This test suite consists the following· - a few Java programs· - a little
160 * helper script to run the java programs - a maven script to build it.··
161 *
162 * When generating data, its best to have each map task generate a multiple of
163 * 25 million. The reason for this is that circular linked list are generated
164 * every 25M. Not generating a multiple in 25M will result in some nodes in the
165 * linked list not having references. The loss of an unreferenced node can not
166 * be detected.
167 *
168 *
169 * Below is a description of the Java programs
170 *
171 * Generator - A map only job that generates data. As stated previously,·its best to generate data
172 * in multiples of 25M. An option is also available to allow concurrent walkers to select and walk
173 * random flushed loops during this phase.
174 *
175 * Verify - A map reduce job that looks for holes. Look at the counts after running. REFERENCED and
176 * UNREFERENCED are· ok, any UNDEFINED counts are bad. Do not run at the· same
177 * time as the Generator.
178 *
179 * Walker - A standalone program that start following a linked list· and emits timing info.··
180 *
181 * Print - A standalone program that prints nodes in the linked list
182 *
183 * Delete - A standalone program that deletes a single node
184 *
185 * This class can be run as a unit test, as an integration test, or from the command line
186 *
187 * ex:
188 * ./hbase org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList
189 *    loop 2 1 100000 /temp 1 1000 50 1 0
190 *
191 */
192@Category(IntegrationTests.class)
193public class IntegrationTestBigLinkedList extends IntegrationTestBase {
194  protected static final byte[] NO_KEY = new byte[1];
195
196  protected static String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
197
198  protected static String DEFAULT_TABLE_NAME = "IntegrationTestBigLinkedList";
199
200  protected static byte[] FAMILY_NAME = Bytes.toBytes("meta");
201  private static byte[] BIG_FAMILY_NAME = Bytes.toBytes("big");
202  private static byte[] TINY_FAMILY_NAME = Bytes.toBytes("tiny");
203
204  //link to the id of the prev node in the linked list
205  protected static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
206
207  //identifier of the mapred task that generated this row
208  protected static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
209
210  //the id of the row within the same client.
211  protected static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
212
213  /** How many rows to write per map task. This has to be a multiple of 25M */
214  private static final String GENERATOR_NUM_ROWS_PER_MAP_KEY
215    = "IntegrationTestBigLinkedList.generator.num_rows";
216
217  private static final String GENERATOR_NUM_MAPPERS_KEY
218    = "IntegrationTestBigLinkedList.generator.map.tasks";
219
220  private static final String GENERATOR_WIDTH_KEY
221    = "IntegrationTestBigLinkedList.generator.width";
222
223  private static final String GENERATOR_WRAP_KEY
224    = "IntegrationTestBigLinkedList.generator.wrap";
225
226  private static final String CONCURRENT_WALKER_KEY
227    = "IntegrationTestBigLinkedList.generator.concurrentwalkers";
228
229  protected int NUM_SLAVES_BASE = 3; // number of slaves for the cluster
230
231  private static final int MISSING_ROWS_TO_LOG = 10; // YARN complains when too many counters
232
233  private static final int WIDTH_DEFAULT = 1000000;
234  private static final int WRAP_DEFAULT = 25;
235  private static final int ROWKEY_LENGTH = 16;
236
237  private static final int CONCURRENT_WALKER_DEFAULT = 0;
238
239  protected String toRun;
240  protected String[] otherArgs;
241
242  static class CINode {
243    byte[] key;
244    byte[] prev;
245    String client;
246    long count;
247  }
248
249  /**
250   * A Map only job that generates random linked list and stores them.
251   */
252  static class Generator extends Configured implements Tool {
253
254    private static final Logger LOG = LoggerFactory.getLogger(Generator.class);
255
256    /**
257     * Set this configuration if you want to test single-column family flush works. If set, we will
258     * add a big column family and a small column family on either side of the usual ITBLL 'meta'
259     * column family. When we write out the ITBLL, we will also add to the big column family a value
260     * bigger than that for ITBLL and for small, something way smaller. The idea is that when
261     * flush-by-column family rather than by region is enabled, we can see if ITBLL is broke in any
262     * way. Here is how you would pass it:
263     * <p>
264     * $ ./bin/hbase org.apache.hadoop.hbase.test.IntegrationTestBigLinkedList
265     * -Dgenerator.multiple.columnfamilies=true generator 1 10 g
266     */
267    public static final String MULTIPLE_UNEVEN_COLUMNFAMILIES_KEY =
268        "generator.multiple.columnfamilies";
269
270    public static enum Counts {
271      SUCCESS, TERMINATING, UNDEFINED, IOEXCEPTION
272    }
273
274    public static final String USAGE =  "Usage : " + Generator.class.getSimpleName() +
275        " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>" +
276        " <num walker threads>] \n" +
277        "where <num nodes per map> should be a multiple of width*wrap multiplier, 25M by default \n" +
278        "walkers will verify random flushed loop during Generation.";
279
280    public Job job;
281
282    static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
283      static class GeneratorInputSplit extends InputSplit implements Writable {
284        @Override
285        public long getLength() throws IOException, InterruptedException {
286          return 1;
287        }
288        @Override
289        public String[] getLocations() throws IOException, InterruptedException {
290          return new String[0];
291        }
292        @Override
293        public void readFields(DataInput arg0) throws IOException {
294        }
295        @Override
296        public void write(DataOutput arg0) throws IOException {
297        }
298      }
299
300      static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
301        private long count;
302        private long numNodes;
303        private Random rand;
304
305        @Override
306        public void close() throws IOException {
307        }
308
309        @Override
310        public BytesWritable getCurrentKey() throws IOException, InterruptedException {
311          byte[] bytes = new byte[ROWKEY_LENGTH];
312          rand.nextBytes(bytes);
313          return new BytesWritable(bytes);
314        }
315
316        @Override
317        public NullWritable getCurrentValue() throws IOException, InterruptedException {
318          return NullWritable.get();
319        }
320
321        @Override
322        public float getProgress() throws IOException, InterruptedException {
323          return (float)(count / (double)numNodes);
324        }
325
326        @Override
327        public void initialize(InputSplit arg0, TaskAttemptContext context)
328            throws IOException, InterruptedException {
329          numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
330          // Use SecureRandom to avoid issue described in HBASE-13382.
331          rand = new SecureRandom();
332        }
333
334        @Override
335        public boolean nextKeyValue() throws IOException, InterruptedException {
336          return count++ < numNodes;
337        }
338
339      }
340
341      @Override
342      public RecordReader<BytesWritable,NullWritable> createRecordReader(
343          InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
344        GeneratorRecordReader rr = new GeneratorRecordReader();
345        rr.initialize(split, context);
346        return rr;
347      }
348
349      @Override
350      public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
351        int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
352
353        ArrayList<InputSplit> splits = new ArrayList<>(numMappers);
354
355        for (int i = 0; i < numMappers; i++) {
356          splits.add(new GeneratorInputSplit());
357        }
358
359        return splits;
360      }
361    }
362
363    /** Ensure output files from prev-job go to map inputs for current job */
364    static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
365      @Override
366      protected boolean isSplitable(JobContext context, Path filename) {
367        return false;
368      }
369    }
370
371    /**
372     * Some ASCII art time:
373     * <p>
374     * [ . . . ] represents one batch of random longs of length WIDTH
375     * <pre>
376     *                _________________________
377     *               |                  ______ |
378     *               |                 |      ||
379     *             .-+-----------------+-----.||
380     *             | |                 |     |||
381     * first   = [ . . . . . . . . . . . ]   |||
382     *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
383     *             | | | | | | | | | | |     |||
384     * prev    = [ . . . . . . . . . . . ]   |||
385     *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
386     *             | | | | | | | | | | |     |||
387     * current = [ . . . . . . . . . . . ]   |||
388     *                                       |||
389     * ...                                   |||
390     *                                       |||
391     * last    = [ . . . . . . . . . . . ]   |||
392     *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^_____|||
393     *             |                 |________||
394     *             |___________________________|
395     * </pre>
396     */
397
398    static class GeneratorMapper
399      extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
400
401      byte[][] first = null;
402      byte[][] prev = null;
403      byte[][] current = null;
404      byte[] id;
405      long count = 0;
406      int i;
407      BufferedMutator mutator;
408      Connection connection;
409      long numNodes;
410      long wrap;
411      int width;
412      boolean multipleUnevenColumnFamilies;
413      byte[] tinyValue = new byte[] { 't' };
414      byte[] bigValue = null;
415      Configuration conf;
416
417      volatile boolean walkersStop;
418      int numWalkers;
419      volatile List<Long> flushedLoops = new ArrayList<>();
420      List<Thread> walkers = new ArrayList<>();
421
422      @Override
423      protected void setup(Context context) throws IOException, InterruptedException {
424        id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID());
425        this.connection = ConnectionFactory.createConnection(context.getConfiguration());
426        instantiateHTable();
427        this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
428        current = new byte[this.width][];
429        int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
430        this.wrap = (long)wrapMultiplier * width;
431        this.numNodes = context.getConfiguration().getLong(
432            GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
433        if (this.numNodes < this.wrap) {
434          this.wrap = this.numNodes;
435        }
436        this.multipleUnevenColumnFamilies = isMultiUnevenColumnFamilies(context.getConfiguration());
437        this.numWalkers = context.getConfiguration().getInt(CONCURRENT_WALKER_KEY, CONCURRENT_WALKER_DEFAULT);
438        this.walkersStop = false;
439        this.conf = context.getConfiguration();
440      }
441
442      protected void instantiateHTable() throws IOException {
443        mutator = connection.getBufferedMutator(
444            new BufferedMutatorParams(getTableName(connection.getConfiguration()))
445                .writeBufferSize(4 * 1024 * 1024));
446      }
447
448      @Override
449      protected void cleanup(Context context) throws IOException ,InterruptedException {
450        joinWalkers();
451        mutator.close();
452        connection.close();
453      }
454
455      @Override
456      protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
457        current[i] = new byte[key.getLength()];
458        System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
459        if (++i == current.length) {
460          LOG.info("Persisting current.length=" + current.length + ", count=" + count + ", id=" +
461            Bytes.toStringBinary(id) + ", current=" + Bytes.toStringBinary(current[0]) +
462            ", i=" + i);
463          persist(output, count, prev, current, id);
464          i = 0;
465
466          if (first == null) {
467            first = current;
468          }
469          prev = current;
470          current = new byte[this.width][];
471
472          count += current.length;
473          output.setStatus("Count " + count);
474
475          if (count % wrap == 0) {
476            // this block of code turns the 1 million linked list of length 25 into one giant
477            //circular linked list of 25 million
478            circularLeftShift(first);
479            persist(output, -1, prev, first, null);
480            // At this point the entire loop has been flushed so we can add one of its nodes to the
481            // concurrent walker
482            if (numWalkers > 0) {
483              addFlushed(key.getBytes());
484              if (walkers.isEmpty()) {
485                startWalkers(numWalkers, conf, output);
486              }
487            }
488            first = null;
489            prev = null;
490          }
491        }
492      }
493
494      private static <T> void circularLeftShift(T[] first) {
495        T ez = first[0];
496        System.arraycopy(first, 1, first, 0, first.length - 1);
497        first[first.length - 1] = ez;
498      }
499
500      private void addFlushed(byte[] rowKey) {
501        synchronized (flushedLoops) {
502          flushedLoops.add(Bytes.toLong(rowKey));
503          flushedLoops.notifyAll();
504        }
505      }
506
507      protected void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
508          throws IOException {
509        for (int i = 0; i < current.length; i++) {
510
511          if (i % 100 == 0) {
512            // Tickle progress every so often else maprunner will think us hung
513            output.progress();
514          }
515
516          Put put = new Put(current[i]);
517          put.addColumn(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
518
519          if (count >= 0) {
520            put.addColumn(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
521          }
522          if (id != null) {
523            put.addColumn(FAMILY_NAME, COLUMN_CLIENT, id);
524          }
525          // See if we are to write multiple columns.
526          if (this.multipleUnevenColumnFamilies) {
527            // Use any column name.
528            put.addColumn(TINY_FAMILY_NAME, TINY_FAMILY_NAME, this.tinyValue);
529            // If we've not allocated bigValue, do it now. Reuse same value each time.
530            if (this.bigValue == null) {
531              this.bigValue = new byte[current[i].length * 10];
532              ThreadLocalRandom.current().nextBytes(this.bigValue);
533            }
534            // Use any column name.
535            put.addColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME, this.bigValue);
536          }
537          mutator.mutate(put);
538        }
539
540        mutator.flush();
541      }
542
543      private void startWalkers(int numWalkers, Configuration conf, Context context) {
544        LOG.info("Starting " + numWalkers + " concurrent walkers");
545        for (int i = 0; i < numWalkers; i++) {
546          Thread walker = new Thread(new ContinuousConcurrentWalker(conf, context));
547          walker.start();
548          walkers.add(walker);
549        }
550      }
551
552      private void joinWalkers() {
553        walkersStop = true;
554        synchronized (flushedLoops) {
555          flushedLoops.notifyAll();
556        }
557        for (Thread walker : walkers) {
558          try {
559            walker.join();
560          } catch (InterruptedException e) {
561            // no-op
562          }
563        }
564      }
565
566      /**
567       * Randomly selects and walks a random flushed loop concurrently with the Generator Mapper by
568       * spawning ConcurrentWalker's with specified StartNodes. These ConcurrentWalker's are
569       * configured to only log erroneous nodes.
570       */
571
572      public class ContinuousConcurrentWalker implements Runnable {
573
574        ConcurrentWalker walker;
575        Configuration conf;
576        Context context;
577        Random rand;
578
579        public ContinuousConcurrentWalker(Configuration conf, Context context) {
580          this.conf = conf;
581          this.context = context;
582          rand = new Random();
583        }
584
585        @Override
586        public void run() {
587          while (!walkersStop) {
588            try {
589              long node = selectLoop();
590              try {
591                walkLoop(node);
592              } catch (IOException e) {
593                context.getCounter(Counts.IOEXCEPTION).increment(1l);
594                return;
595              }
596            } catch (InterruptedException e) {
597              return;
598            }
599          }
600        }
601
602        private void walkLoop(long node) throws IOException {
603          walker = new ConcurrentWalker(context);
604          walker.setConf(conf);
605          walker.run(node, wrap);
606        }
607
608        private long selectLoop () throws InterruptedException{
609          synchronized (flushedLoops) {
610            while (flushedLoops.isEmpty() && !walkersStop) {
611              flushedLoops.wait();
612            }
613            if (walkersStop) {
614              throw new InterruptedException();
615            }
616            return flushedLoops.get(rand.nextInt(flushedLoops.size()));
617          }
618        }
619      }
620
621      public static class ConcurrentWalker extends WalkerBase {
622
623        Context context;
624
625        public ConcurrentWalker(Context context) {this.context = context;}
626
627        public void run(long startKeyIn, long maxQueriesIn) throws IOException {
628
629          long maxQueries = maxQueriesIn > 0 ? maxQueriesIn : Long.MAX_VALUE;
630          byte[] startKey = Bytes.toBytes(startKeyIn);
631
632          Connection connection = ConnectionFactory.createConnection(getConf());
633          Table table = connection.getTable(getTableName(getConf()));
634          long numQueries = 0;
635          // If isSpecificStart is set, only walk one list from that particular node.
636          // Note that in case of circular (or P-shaped) list it will walk forever, as is
637          // the case in normal run without startKey.
638
639          CINode node = findStartNode(table, startKey);
640          if (node == null) {
641            LOG.error("Start node not found: " + Bytes.toStringBinary(startKey));
642            throw new IOException("Start node not found: " + startKeyIn);
643          }
644          while (numQueries < maxQueries) {
645            numQueries++;
646            byte[] prev = node.prev;
647            long t1 = System.currentTimeMillis();
648            node = getNode(prev, table, node);
649            long t2 = System.currentTimeMillis();
650            if (node == null) {
651              LOG.error("ConcurrentWalker found UNDEFINED NODE: " + Bytes.toStringBinary(prev));
652              context.getCounter(Counts.UNDEFINED).increment(1l);
653            } else if (node.prev.length == NO_KEY.length) {
654              LOG.error("ConcurrentWalker found TERMINATING NODE: " +
655                  Bytes.toStringBinary(node.key));
656              context.getCounter(Counts.TERMINATING).increment(1l);
657            } else {
658              // Increment for successful walk
659              context.getCounter(Counts.SUCCESS).increment(1l);
660            }
661          }
662          table.close();
663          connection.close();
664        }
665      }
666    }
667
668    @Override
669    public int run(String[] args) throws Exception {
670      if (args.length < 3) {
671        System.err.println(USAGE);
672        return 1;
673      }
674      try {
675        int numMappers = Integer.parseInt(args[0]);
676        long numNodes = Long.parseLong(args[1]);
677        Path tmpOutput = new Path(args[2]);
678        Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
679        Integer wrapMultiplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
680        Integer numWalkers = (args.length < 6) ? null : Integer.parseInt(args[5]);
681        return run(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
682      } catch (NumberFormatException e) {
683        System.err.println("Parsing generator arguments failed: " + e.getMessage());
684        System.err.println(USAGE);
685        return 1;
686      }
687    }
688
689    protected void createSchema() throws IOException {
690      Configuration conf = getConf();
691      TableName tableName = getTableName(conf);
692      try (Connection conn = ConnectionFactory.createConnection(conf);
693          Admin admin = conn.getAdmin()) {
694        if (!admin.tableExists(tableName)) {
695          HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
696          htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
697          // Always add these families. Just skip writing to them when we do not test per CF flush.
698          htd.addFamily(new HColumnDescriptor(BIG_FAMILY_NAME));
699          htd.addFamily(new HColumnDescriptor(TINY_FAMILY_NAME));
700          // if -DuseMob=true force all data through mob path.
701          if (conf.getBoolean("useMob", false)) {
702            for (HColumnDescriptor hcd : htd.getColumnFamilies() ) {
703              hcd.setMobEnabled(true);
704              hcd.setMobThreshold(4);
705            }
706          }
707
708          // If we want to pre-split compute how many splits.
709          if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
710              HBaseTestingUtility.PRESPLIT_TEST_TABLE)) {
711            int numberOfServers =
712                admin.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
713                    .getLiveServerMetrics().size();
714            if (numberOfServers == 0) {
715              throw new IllegalStateException("No live regionservers");
716            }
717            int regionsPerServer = conf.getInt(HBaseTestingUtility.REGIONS_PER_SERVER_KEY,
718                HBaseTestingUtility.DEFAULT_REGIONS_PER_SERVER);
719            int totalNumberOfRegions = numberOfServers * regionsPerServer;
720            LOG.info("Number of live regionservers: " + numberOfServers + ", " +
721                "pre-splitting table into " + totalNumberOfRegions + " regions " +
722                "(default regions per server: " + regionsPerServer + ")");
723
724
725            byte[][] splits = new RegionSplitter.UniformSplit().split(totalNumberOfRegions);
726
727            admin.createTable(htd, splits);
728          } else {
729            // Looks like we're just letting things play out.
730            // Create a table with on region by default.
731            // This will make the splitting work hard.
732            admin.createTable(htd);
733          }
734        }
735      } catch (MasterNotRunningException e) {
736        LOG.error("Master not running", e);
737        throw new IOException(e);
738      }
739    }
740
741    public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
742        Integer width, Integer wrapMultiplier, Integer numWalkers)
743        throws Exception {
744      LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
745          + ", numNodes=" + numNodes);
746      Job job = Job.getInstance(getConf());
747
748      job.setJobName("Random Input Generator");
749      job.setNumReduceTasks(0);
750      job.setJarByClass(getClass());
751
752      job.setInputFormatClass(GeneratorInputFormat.class);
753      job.setOutputKeyClass(BytesWritable.class);
754      job.setOutputValueClass(NullWritable.class);
755
756      setJobConf(job, numMappers, numNodes, width, wrapMultiplier, numWalkers);
757
758      job.setMapperClass(Mapper.class); //identity mapper
759
760      FileOutputFormat.setOutputPath(job, tmpOutput);
761      job.setOutputFormatClass(SequenceFileOutputFormat.class);
762
763      boolean success = jobCompletion(job);
764
765      return success ? 0 : 1;
766    }
767
768    public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
769        Integer width, Integer wrapMultiplier, Integer numWalkers)
770        throws Exception {
771      LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
772      createSchema();
773      job = Job.getInstance(getConf());
774
775      job.setJobName("Link Generator");
776      job.setNumReduceTasks(0);
777      job.setJarByClass(getClass());
778
779      FileInputFormat.setInputPaths(job, tmpOutput);
780      job.setInputFormatClass(OneFilePerMapperSFIF.class);
781      job.setOutputKeyClass(NullWritable.class);
782      job.setOutputValueClass(NullWritable.class);
783
784      setJobConf(job, numMappers, numNodes, width, wrapMultiplier, numWalkers);
785
786      setMapperForGenerator(job);
787
788      job.setOutputFormatClass(NullOutputFormat.class);
789
790      job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
791      TableMapReduceUtil.addDependencyJars(job);
792      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
793                                                     AbstractHBaseTool.class);
794      TableMapReduceUtil.initCredentials(job);
795
796      boolean success = jobCompletion(job);
797
798      return success ? 0 : 1;
799    }
800
801    protected boolean jobCompletion(Job job) throws IOException, InterruptedException,
802        ClassNotFoundException {
803      boolean success = job.waitForCompletion(true);
804      return success;
805    }
806
807    protected void setMapperForGenerator(Job job) {
808      job.setMapperClass(GeneratorMapper.class);
809    }
810
811    public int run(int numMappers, long numNodes, Path tmpOutput,
812        Integer width, Integer wrapMultiplier, Integer numWalkers)
813        throws Exception {
814      int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMultiplier,
815          numWalkers);
816      if (ret > 0) {
817        return ret;
818      }
819      return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
820    }
821
822    public boolean verify() {
823      try {
824        Counters counters = job.getCounters();
825        if (counters == null) {
826          LOG.info("Counters object was null, Generator verification cannot be performed."
827              + " This is commonly a result of insufficient YARN configuration.");
828          return false;
829        }
830
831        if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
832            counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
833            counters.findCounter(Counts.IOEXCEPTION).getValue() > 0) {
834          LOG.error("Concurrent walker failed to verify during Generation phase");
835          LOG.error("TERMINATING nodes: " + counters.findCounter(Counts.TERMINATING).getValue());
836          LOG.error("UNDEFINED nodes: " + counters.findCounter(Counts.UNDEFINED).getValue());
837          LOG.error("IOEXCEPTION nodes: " + counters.findCounter(Counts.IOEXCEPTION).getValue());
838          return false;
839        }
840      } catch (IOException e) {
841        LOG.info("Generator verification could not find counter");
842        return false;
843      }
844      return true;
845    }
846  }
847
848  /**
849   * Tool to search missing rows in WALs and hfiles.
850   * Pass in file or dir of keys to search for. Key file must have been written by Verify step
851   * (we depend on the format it writes out. We'll read them in and then search in hbase
852   * WALs and oldWALs dirs (Some of this is TODO).
853   */
854  static class Search extends Configured implements Tool {
855    private static final Logger LOG = LoggerFactory.getLogger(Search.class);
856    protected Job job;
857
858    private static void printUsage(final String error) {
859      if (error != null && error.length() > 0) System.out.println("ERROR: " + error);
860      System.err.println("Usage: search <KEYS_DIR> [<MAPPERS_COUNT>]");
861    }
862
863    @Override
864    public int run(String[] args) throws Exception {
865      if (args.length < 1 || args.length > 2) {
866        printUsage(null);
867        return 1;
868      }
869      Path inputDir = new Path(args[0]);
870      int numMappers = 1;
871      if (args.length > 1) {
872        numMappers = Integer.parseInt(args[1]);
873      }
874      return run(inputDir, numMappers);
875    }
876
877    /**
878     * WALPlayer override that searches for keys loaded in the setup.
879     */
880    public static class WALSearcher extends WALPlayer {
881      public WALSearcher(Configuration conf) {
882        super(conf);
883      }
884
885      /**
886       * The actual searcher mapper.
887       */
888      public static class WALMapperSearcher extends WALMapper {
889        private SortedSet<byte []> keysToFind;
890        private AtomicInteger rows = new AtomicInteger(0);
891
892        @Override
893        public void setup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation>.Context context)
894            throws IOException {
895          super.setup(context);
896          try {
897            this.keysToFind = readKeysToSearch(context.getConfiguration());
898            LOG.info("Loaded keys to find: count=" + this.keysToFind.size());
899          } catch (InterruptedException e) {
900            throw new InterruptedIOException(e.toString());
901          }
902        }
903
904        @Override
905        protected boolean filter(Context context, Cell cell) {
906          // TODO: Can I do a better compare than this copying out key?
907          byte [] row = new byte [cell.getRowLength()];
908          System.arraycopy(cell.getRowArray(), cell.getRowOffset(), row, 0, cell.getRowLength());
909          boolean b = this.keysToFind.contains(row);
910          if (b) {
911            String keyStr = Bytes.toStringBinary(row);
912            try {
913              LOG.info("Found cell=" + cell + " , walKey=" + context.getCurrentKey());
914            } catch (IOException|InterruptedException e) {
915              LOG.warn(e.toString(), e);
916            }
917            if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
918              context.getCounter(FOUND_GROUP_KEY, keyStr).increment(1);
919            }
920            context.getCounter(FOUND_GROUP_KEY, "CELL_WITH_MISSING_ROW").increment(1);
921          }
922          return b;
923        }
924      }
925
926      // Put in place the above WALMapperSearcher.
927      @Override
928      public Job createSubmittableJob(String[] args) throws IOException {
929        Job job = super.createSubmittableJob(args);
930        // Call my class instead.
931        job.setJarByClass(WALMapperSearcher.class);
932        job.setMapperClass(WALMapperSearcher.class);
933        job.setOutputFormatClass(NullOutputFormat.class);
934        return job;
935      }
936    }
937
938    static final String FOUND_GROUP_KEY = "Found";
939    static final String SEARCHER_INPUTDIR_KEY = "searcher.keys.inputdir";
940
941    public int run(Path inputDir, int numMappers) throws Exception {
942      getConf().set(SEARCHER_INPUTDIR_KEY, inputDir.toString());
943      SortedSet<byte []> keys = readKeysToSearch(getConf());
944      if (keys.isEmpty()) throw new RuntimeException("No keys to find");
945      LOG.info("Count of keys to find: " + keys.size());
946      for(byte [] key: keys)  LOG.info("Key: " + Bytes.toStringBinary(key));
947      Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
948      // Now read all WALs. In two dirs. Presumes certain layout.
949      Path walsDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
950      Path oldWalsDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
951      LOG.info("Running Search with keys inputDir=" + inputDir +", numMappers=" + numMappers +
952        " against " + getConf().get(HConstants.HBASE_DIR));
953      int ret = ToolRunner.run(getConf(), new WALSearcher(getConf()),
954          new String [] {walsDir.toString(), ""});
955      if (ret != 0) {
956        return ret;
957      }
958      return ToolRunner.run(getConf(), new WALSearcher(getConf()),
959          new String [] {oldWalsDir.toString(), ""});
960    }
961
962    static SortedSet<byte []> readKeysToSearch(final Configuration conf)
963    throws IOException, InterruptedException {
964      Path keysInputDir = new Path(conf.get(SEARCHER_INPUTDIR_KEY));
965      FileSystem fs = FileSystem.get(conf);
966      SortedSet<byte []> result = new TreeSet<>(Bytes.BYTES_COMPARATOR);
967      if (!fs.exists(keysInputDir)) {
968        throw new FileNotFoundException(keysInputDir.toString());
969      }
970      if (!fs.isDirectory(keysInputDir)) {
971        throw new UnsupportedOperationException("TODO");
972      } else {
973        RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(keysInputDir, false);
974        while(iterator.hasNext()) {
975          LocatedFileStatus keyFileStatus = iterator.next();
976          // Skip "_SUCCESS" file.
977          if (keyFileStatus.getPath().getName().startsWith("_")) continue;
978          result.addAll(readFileToSearch(conf, fs, keyFileStatus));
979        }
980      }
981      return result;
982    }
983
984    private static SortedSet<byte[]> readFileToSearch(final Configuration conf,
985        final FileSystem fs, final LocatedFileStatus keyFileStatus) throws IOException,
986        InterruptedException {
987      SortedSet<byte []> result = new TreeSet<>(Bytes.BYTES_COMPARATOR);
988      // Return entries that are flagged Counts.UNDEFINED in the value. Return the row. This is
989      // what is missing.
990      TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
991      try (SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader rr =
992          new SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader()) {
993        InputSplit is =
994          new FileSplit(keyFileStatus.getPath(), 0, keyFileStatus.getLen(), new String [] {});
995        rr.initialize(is, context);
996        while (rr.nextKeyValue()) {
997          rr.getCurrentKey();
998          BytesWritable bw = rr.getCurrentValue();
999          if (Verify.VerifyReducer.whichType(bw.getBytes()) == Verify.Counts.UNDEFINED) {
1000            byte[] key = new byte[rr.getCurrentKey().getLength()];
1001            System.arraycopy(rr.getCurrentKey().getBytes(), 0, key, 0, rr.getCurrentKey()
1002                .getLength());
1003            result.add(key);
1004          }
1005        }
1006      }
1007      return result;
1008    }
1009  }
1010
1011  /**
1012   * A Map Reduce job that verifies that the linked lists generated by
1013   * {@link Generator} do not have any holes.
1014   */
1015  static class Verify extends Configured implements Tool {
1016
1017    private static final Logger LOG = LoggerFactory.getLogger(Verify.class);
1018    protected static final BytesWritable DEF = new BytesWritable(new byte[] { 0 });
1019    protected static final BytesWritable DEF_LOST_FAMILIES = new BytesWritable(new byte[] { 1 });
1020
1021    protected Job job;
1022
1023    public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
1024      private BytesWritable row = new BytesWritable();
1025      private BytesWritable ref = new BytesWritable();
1026
1027      private boolean multipleUnevenColumnFamilies;
1028
1029      @Override
1030      protected void setup(
1031          Mapper<ImmutableBytesWritable, Result, BytesWritable, BytesWritable>.Context context)
1032          throws IOException, InterruptedException {
1033        this.multipleUnevenColumnFamilies = isMultiUnevenColumnFamilies(context.getConfiguration());
1034      }
1035
1036      @Override
1037      protected void map(ImmutableBytesWritable key, Result value, Context context)
1038          throws IOException ,InterruptedException {
1039        byte[] rowKey = key.get();
1040        row.set(rowKey, 0, rowKey.length);
1041        if (multipleUnevenColumnFamilies
1042            && (!value.containsColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME) || !value.containsColumn(
1043              TINY_FAMILY_NAME, TINY_FAMILY_NAME))) {
1044          context.write(row, DEF_LOST_FAMILIES);
1045        } else {
1046          context.write(row, DEF);
1047        }
1048        byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
1049        if (prev != null && prev.length > 0) {
1050          ref.set(prev, 0, prev.length);
1051          context.write(ref, row);
1052        } else {
1053          LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
1054        }
1055      }
1056    }
1057
1058    /**
1059     * Don't change the order of these enums. Their ordinals are used as type flag when we emit
1060     * problems found from the reducer.
1061     */
1062    public static enum Counts {
1063      UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES, EXTRA_UNDEF_REFERENCES,
1064      LOST_FAMILIES
1065    }
1066
1067    /**
1068     * Per reducer, we output problem rows as byte arrasy so can be used as input for
1069     * subsequent investigative mapreduce jobs. Each emitted value is prefaced by a one byte flag
1070     * saying what sort of emission it is. Flag is the Count enum ordinal as a short.
1071     */
1072    public static class VerifyReducer extends
1073        Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable> {
1074      private ArrayList<byte[]> refs = new ArrayList<>();
1075      private final BytesWritable UNREF = new BytesWritable(addPrefixFlag(
1076        Counts.UNREFERENCED.ordinal(), new byte[] {}));
1077      private final BytesWritable LOSTFAM = new BytesWritable(addPrefixFlag(
1078        Counts.LOST_FAMILIES.ordinal(), new byte[] {}));
1079
1080      private AtomicInteger rows = new AtomicInteger(0);
1081      private Connection connection;
1082
1083      @Override
1084      protected void setup(Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
1085      throws IOException, InterruptedException {
1086        super.setup(context);
1087        this.connection = ConnectionFactory.createConnection(context.getConfiguration());
1088      }
1089
1090      @Override
1091      protected void cleanup(
1092          Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
1093          throws IOException, InterruptedException {
1094        if (this.connection != null) {
1095          this.connection.close();
1096        }
1097        super.cleanup(context);
1098      }
1099
1100      /**
1101       * @param ordinal
1102       * @param r
1103       * @return Return new byte array that has <code>ordinal</code> as prefix on front taking up
1104       * Bytes.SIZEOF_SHORT bytes followed by <code>r</code>
1105       */
1106      public static byte[] addPrefixFlag(final int ordinal, final byte [] r) {
1107        byte[] prefix = Bytes.toBytes((short)ordinal);
1108        if (prefix.length != Bytes.SIZEOF_SHORT) {
1109          throw new RuntimeException("Unexpected size: " + prefix.length);
1110        }
1111        byte[] result = new byte[prefix.length + r.length];
1112        System.arraycopy(prefix, 0, result, 0, prefix.length);
1113        System.arraycopy(r, 0, result, prefix.length, r.length);
1114        return result;
1115      }
1116
1117      /**
1118       * @param bs
1119       * @return Type from the Counts enum of this row. Reads prefix added by
1120       * {@link #addPrefixFlag(int, byte[])}
1121       */
1122      public static Counts whichType(final byte [] bs) {
1123        int ordinal = Bytes.toShort(bs, 0, Bytes.SIZEOF_SHORT);
1124        return Counts.values()[ordinal];
1125      }
1126
1127      /**
1128       * @param bw
1129       * @return Row bytes minus the type flag.
1130       */
1131      public static byte[] getRowOnly(BytesWritable bw) {
1132        byte[] bytes = new byte [bw.getLength() - Bytes.SIZEOF_SHORT];
1133        System.arraycopy(bw.getBytes(), Bytes.SIZEOF_SHORT, bytes, 0, bytes.length);
1134        return bytes;
1135      }
1136
1137      @Override
1138      public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
1139          throws IOException, InterruptedException {
1140        int defCount = 0;
1141        boolean lostFamilies = false;
1142        refs.clear();
1143        for (BytesWritable type : values) {
1144          if (type.getLength() == DEF.getLength()) {
1145            defCount++;
1146            if (type.getBytes()[0] == 1) {
1147              lostFamilies = true;
1148            }
1149          } else {
1150            byte[] bytes = new byte[type.getLength()];
1151            System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
1152            refs.add(bytes);
1153          }
1154        }
1155
1156        // TODO check for more than one def, should not happen
1157        StringBuilder refsSb = null;
1158        String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
1159        if (defCount == 0 || refs.size() != 1) {
1160          refsSb = dumpExtraInfoOnRefs(key, context, refs);
1161          LOG.error("LinkedListError: key=" + keyString + ", reference(s)=" +
1162            (refsSb != null? refsSb.toString(): ""));
1163        }
1164        if (lostFamilies) {
1165          LOG.error("LinkedListError: key=" + keyString + ", lost big or tiny families");
1166          context.getCounter(Counts.LOST_FAMILIES).increment(1);
1167          context.write(key, LOSTFAM);
1168        }
1169
1170        if (defCount == 0 && refs.size() > 0) {
1171          // This is bad, found a node that is referenced but not defined. It must have been
1172          // lost, emit some info about this node for debugging purposes.
1173          // Write out a line per reference. If more than one, flag it.;
1174          for (int i = 0; i < refs.size(); i++) {
1175            byte[] bs = refs.get(i);
1176            int ordinal;
1177            if (i <= 0) {
1178              ordinal = Counts.UNDEFINED.ordinal();
1179              context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
1180              context.getCounter(Counts.UNDEFINED).increment(1);
1181            } else {
1182              ordinal = Counts.EXTRA_UNDEF_REFERENCES.ordinal();
1183              context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
1184            }
1185          }
1186          if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
1187            // Print out missing row; doing get on reference gives info on when the referencer
1188            // was added which can help a little debugging. This info is only available in mapper
1189            // output -- the 'Linked List error Key...' log message above. What we emit here is
1190            // useless for debugging.
1191            context.getCounter("undef", keyString).increment(1);
1192          }
1193        } else if (defCount > 0 && refs.isEmpty()) {
1194          // node is defined but not referenced
1195          context.write(key, UNREF);
1196          context.getCounter(Counts.UNREFERENCED).increment(1);
1197          if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
1198            context.getCounter("unref", keyString).increment(1);
1199          }
1200        } else {
1201          if (refs.size() > 1) {
1202            // Skip first reference.
1203            for (int i = 1; i < refs.size(); i++) {
1204              context.write(key,
1205                new BytesWritable(addPrefixFlag(Counts.EXTRAREFERENCES.ordinal(), refs.get(i))));
1206            }
1207            context.getCounter(Counts.EXTRAREFERENCES).increment(refs.size() - 1);
1208          }
1209          // node is defined and referenced
1210          context.getCounter(Counts.REFERENCED).increment(1);
1211        }
1212      }
1213
1214      /**
1215       * Dump out extra info around references if there are any. Helps debugging.
1216       * @return StringBuilder filled with references if any.
1217       * @throws IOException
1218       */
1219      private StringBuilder dumpExtraInfoOnRefs(final BytesWritable key, final Context context,
1220          final List<byte []> refs)
1221      throws IOException {
1222        StringBuilder refsSb = null;
1223        if (refs.isEmpty()) return refsSb;
1224        refsSb = new StringBuilder();
1225        String comma = "";
1226        // If a row is a reference but has no define, print the content of the row that has
1227        // this row as a 'prev'; it will help debug.  The missing row was written just before
1228        // the row we are dumping out here.
1229        TableName tn = getTableName(context.getConfiguration());
1230        try (Table t = this.connection.getTable(tn)) {
1231          for (byte [] ref : refs) {
1232            Result r = t.get(new Get(ref));
1233            List<Cell> cells = r.listCells();
1234            String ts = (cells != null && !cells.isEmpty())?
1235                new java.util.Date(cells.get(0).getTimestamp()).toString(): "";
1236            byte [] b = r.getValue(FAMILY_NAME, COLUMN_CLIENT);
1237            String jobStr = (b != null && b.length > 0)? Bytes.toString(b): "";
1238            b = r.getValue(FAMILY_NAME, COLUMN_COUNT);
1239            long count = (b != null && b.length > 0)? Bytes.toLong(b): -1;
1240            b = r.getValue(FAMILY_NAME, COLUMN_PREV);
1241            String refRegionLocation = "";
1242            String keyRegionLocation = "";
1243            if (b != null && b.length > 0) {
1244              try (RegionLocator rl = this.connection.getRegionLocator(tn)) {
1245                HRegionLocation hrl = rl.getRegionLocation(b);
1246                if (hrl != null) refRegionLocation = hrl.toString();
1247                // Key here probably has trailing zeros on it.
1248                hrl = rl.getRegionLocation(key.getBytes());
1249                if (hrl != null) keyRegionLocation = hrl.toString();
1250              }
1251            }
1252            LOG.error("Extras on ref without a def, ref=" + Bytes.toStringBinary(ref) +
1253              ", refPrevEqualsKey=" +
1254                (Bytes.compareTo(key.getBytes(), 0, key.getLength(), b, 0, b.length) == 0) +
1255                ", key=" + Bytes.toStringBinary(key.getBytes(), 0, key.getLength()) +
1256                ", ref row date=" + ts + ", jobStr=" + jobStr +
1257                ", ref row count=" + count +
1258                ", ref row regionLocation=" + refRegionLocation +
1259                ", key row regionLocation=" + keyRegionLocation);
1260            refsSb.append(comma);
1261            comma = ",";
1262            refsSb.append(Bytes.toStringBinary(ref));
1263          }
1264        }
1265        return refsSb;
1266      }
1267    }
1268
1269    @Override
1270    public int run(String[] args) throws Exception {
1271      if (args.length != 2) {
1272        System.out.println("Usage : " + Verify.class.getSimpleName()
1273            + " <output dir> <num reducers>");
1274        return 0;
1275      }
1276
1277      String outputDir = args[0];
1278      int numReducers = Integer.parseInt(args[1]);
1279
1280      return run(outputDir, numReducers);
1281    }
1282
1283    public int run(String outputDir, int numReducers) throws Exception {
1284      return run(new Path(outputDir), numReducers);
1285    }
1286
1287    public int run(Path outputDir, int numReducers) throws Exception {
1288      LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
1289
1290      job = Job.getInstance(getConf());
1291
1292      job.setJobName("Link Verifier");
1293      job.setNumReduceTasks(numReducers);
1294      job.setJarByClass(getClass());
1295
1296      setJobScannerConf(job);
1297
1298      Scan scan = new Scan();
1299      scan.addColumn(FAMILY_NAME, COLUMN_PREV);
1300      scan.setCaching(10000);
1301      scan.setCacheBlocks(false);
1302      if (isMultiUnevenColumnFamilies(getConf())) {
1303        scan.addColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME);
1304        scan.addColumn(TINY_FAMILY_NAME, TINY_FAMILY_NAME);
1305      }
1306
1307      TableMapReduceUtil.initTableMapperJob(getTableName(getConf()).getName(), scan,
1308          VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
1309      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
1310                                                     AbstractHBaseTool.class);
1311
1312      job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
1313
1314      job.setReducerClass(VerifyReducer.class);
1315      job.setOutputFormatClass(SequenceFileAsBinaryOutputFormat.class);
1316      job.setOutputKeyClass(BytesWritable.class);
1317      job.setOutputValueClass(BytesWritable.class);
1318      TextOutputFormat.setOutputPath(job, outputDir);
1319
1320      boolean success = job.waitForCompletion(true);
1321
1322      if (success) {
1323        Counters counters = job.getCounters();
1324        if (null == counters) {
1325          LOG.warn("Counters were null, cannot verify Job completion."
1326              + " This is commonly a result of insufficient YARN configuration.");
1327          // We don't have access to the counters to know if we have "bad" counts
1328          return 0;
1329        }
1330
1331        // If we find no unexpected values, the job didn't outright fail
1332        if (verifyUnexpectedValues(counters)) {
1333          // We didn't check referenced+unreferenced counts, leave that to visual inspection
1334          return 0;
1335        }
1336      }
1337
1338      // We failed
1339      return 1;
1340    }
1341
1342    public boolean verify(long expectedReferenced) throws Exception {
1343      if (job == null) {
1344        throw new IllegalStateException("You should call run() first");
1345      }
1346
1347      Counters counters = job.getCounters();
1348      if (counters == null) {
1349        LOG.info("Counters object was null, write verification cannot be performed."
1350              + " This is commonly a result of insufficient YARN configuration.");
1351        return false;
1352      }
1353
1354      // Run through each check, even if we fail one early
1355      boolean success = verifyExpectedValues(expectedReferenced, counters);
1356
1357      if (!verifyUnexpectedValues(counters)) {
1358        // We found counter objects which imply failure
1359        success = false;
1360      }
1361
1362      if (!success) {
1363        handleFailure(counters);
1364      }
1365      return success;
1366    }
1367
1368    /**
1369     * Verify the values in the Counters against the expected number of entries written.
1370     *
1371     * @param expectedReferenced
1372     *          Expected number of referenced entrires
1373     * @param counters
1374     *          The Job's Counters object
1375     * @return True if the values match what's expected, false otherwise
1376     */
1377    protected boolean verifyExpectedValues(long expectedReferenced, Counters counters) {
1378      final Counter referenced = counters.findCounter(Counts.REFERENCED);
1379      final Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
1380      boolean success = true;
1381
1382      if (expectedReferenced != referenced.getValue()) {
1383        LOG.error("Expected referenced count does not match with actual referenced count. " +
1384            "expected referenced=" + expectedReferenced + " ,actual=" + referenced.getValue());
1385        success = false;
1386      }
1387
1388      if (unreferenced.getValue() > 0) {
1389        final Counter multiref = counters.findCounter(Counts.EXTRAREFERENCES);
1390        boolean couldBeMultiRef = (multiref.getValue() == unreferenced.getValue());
1391        LOG.error("Unreferenced nodes were not expected. Unreferenced count=" + unreferenced.getValue()
1392            + (couldBeMultiRef ? "; could be due to duplicate random numbers" : ""));
1393        success = false;
1394      }
1395
1396      return success;
1397    }
1398
1399    /**
1400     * Verify that the Counters don't contain values which indicate an outright failure from the Reducers.
1401     *
1402     * @param counters
1403     *          The Job's counters
1404     * @return True if the "bad" counter objects are 0, false otherwise
1405     */
1406    protected boolean verifyUnexpectedValues(Counters counters) {
1407      final Counter undefined = counters.findCounter(Counts.UNDEFINED);
1408      final Counter lostfamilies = counters.findCounter(Counts.LOST_FAMILIES);
1409      boolean success = true;
1410
1411      if (undefined.getValue() > 0) {
1412        LOG.error("Found an undefined node. Undefined count=" + undefined.getValue());
1413        success = false;
1414      }
1415
1416      if (lostfamilies.getValue() > 0) {
1417        LOG.error("Found nodes which lost big or tiny families, count=" + lostfamilies.getValue());
1418        success = false;
1419      }
1420
1421      return success;
1422    }
1423
1424    protected void handleFailure(Counters counters) throws IOException {
1425      Configuration conf = job.getConfiguration();
1426      TableName tableName = getTableName(conf);
1427      try (Connection conn = ConnectionFactory.createConnection(conf)) {
1428        try (RegionLocator rl = conn.getRegionLocator(tableName)) {
1429          CounterGroup g = counters.getGroup("undef");
1430          Iterator<Counter> it = g.iterator();
1431          while (it.hasNext()) {
1432            String keyString = it.next().getName();
1433            byte[] key = Bytes.toBytes(keyString);
1434            HRegionLocation loc = rl.getRegionLocation(key, true);
1435            LOG.error("undefined row " + keyString + ", " + loc);
1436          }
1437          g = counters.getGroup("unref");
1438          it = g.iterator();
1439          while (it.hasNext()) {
1440            String keyString = it.next().getName();
1441            byte[] key = Bytes.toBytes(keyString);
1442            HRegionLocation loc = rl.getRegionLocation(key, true);
1443            LOG.error("unreferred row " + keyString + ", " + loc);
1444          }
1445        }
1446      }
1447    }
1448  }
1449
1450  /**
1451   * Executes Generate and Verify in a loop. Data is not cleaned between runs, so each iteration
1452   * adds more data.
1453   */
1454  static class Loop extends Configured implements Tool {
1455
1456    private static final Logger LOG = LoggerFactory.getLogger(Loop.class);
1457    private static final String USAGE = "Usage: Loop <num iterations> <num mappers> " +
1458        "<num nodes per mapper> <output dir> <num reducers> [<width> <wrap multiplier>" +
1459        " <num walker threads>] \n" +
1460        "where <num nodes per map> should be a multiple of width*wrap multiplier, 25M by default \n" +
1461        "walkers will select and verify random flushed loop during Generation.";
1462
1463    IntegrationTestBigLinkedList it;
1464
1465    protected void runGenerator(int numMappers, long numNodes,
1466        String outputDir, Integer width, Integer wrapMultiplier, Integer numWalkers)
1467        throws Exception {
1468      Path outputPath = new Path(outputDir);
1469      UUID uuid = UUID.randomUUID(); //create a random UUID.
1470      Path generatorOutput = new Path(outputPath, uuid.toString());
1471
1472      Generator generator = new Generator();
1473      generator.setConf(getConf());
1474      int retCode = generator.run(numMappers, numNodes, generatorOutput, width, wrapMultiplier,
1475          numWalkers);
1476      if (retCode > 0) {
1477        throw new RuntimeException("Generator failed with return code: " + retCode);
1478      }
1479      if (numWalkers > 0) {
1480        if (!generator.verify()) {
1481          throw new RuntimeException("Generator.verify failed");
1482        }
1483      }
1484    }
1485
1486    protected void runVerify(String outputDir,
1487        int numReducers, long expectedNumNodes) throws Exception {
1488      Path outputPath = new Path(outputDir);
1489      UUID uuid = UUID.randomUUID(); //create a random UUID.
1490      Path iterationOutput = new Path(outputPath, uuid.toString());
1491
1492      Verify verify = new Verify();
1493      verify.setConf(getConf());
1494      int retCode = verify.run(iterationOutput, numReducers);
1495      if (retCode > 0) {
1496        throw new RuntimeException("Verify.run failed with return code: " + retCode);
1497      }
1498
1499      if (!verify.verify(expectedNumNodes)) {
1500        throw new RuntimeException("Verify.verify failed");
1501      }
1502      LOG.info("Verify finished with success. Total nodes=" + expectedNumNodes);
1503    }
1504
1505    @Override
1506    public int run(String[] args) throws Exception {
1507      if (args.length < 5) {
1508        System.err.println(USAGE);
1509        return 1;
1510      }
1511      try {
1512        int numIterations = Integer.parseInt(args[0]);
1513        int numMappers = Integer.parseInt(args[1]);
1514        long numNodes = Long.parseLong(args[2]);
1515        String outputDir = args[3];
1516        int numReducers = Integer.parseInt(args[4]);
1517        Integer width = (args.length < 6) ? null : Integer.parseInt(args[5]);
1518        Integer wrapMultiplier = (args.length < 7) ? null : Integer.parseInt(args[6]);
1519        Integer numWalkers = (args.length < 8) ? 0 : Integer.parseInt(args[7]);
1520
1521        long expectedNumNodes = 0;
1522
1523        if (numIterations < 0) {
1524          numIterations = Integer.MAX_VALUE; //run indefinitely (kind of)
1525        }
1526        LOG.info("Running Loop with args:" + Arrays.deepToString(args));
1527        for (int i = 0; i < numIterations; i++) {
1528          LOG.info("Starting iteration = " + i);
1529          runGenerator(numMappers, numNodes, outputDir, width, wrapMultiplier, numWalkers);
1530          expectedNumNodes += numMappers * numNodes;
1531          runVerify(outputDir, numReducers, expectedNumNodes);
1532        }
1533        return 0;
1534      } catch (NumberFormatException e) {
1535        System.err.println("Parsing loop arguments failed: " + e.getMessage());
1536        System.err.println(USAGE);
1537        return 1;
1538      }
1539    }
1540  }
1541
1542  /**
1543   * A stand alone program that prints out portions of a list created by {@link Generator}
1544   */
1545  private static class Print extends Configured implements Tool {
1546    @Override
1547    public int run(String[] args) throws Exception {
1548      Options options = new Options();
1549      options.addOption("s", "start", true, "start key");
1550      options.addOption("e", "end", true, "end key");
1551      options.addOption("l", "limit", true, "number to print");
1552
1553      GnuParser parser = new GnuParser();
1554      CommandLine cmd = null;
1555      try {
1556        cmd = parser.parse(options, args);
1557        if (cmd.getArgs().length != 0) {
1558          throw new ParseException("Command takes no arguments");
1559        }
1560      } catch (ParseException e) {
1561        System.err.println("Failed to parse command line " + e.getMessage());
1562        System.err.println();
1563        HelpFormatter formatter = new HelpFormatter();
1564        formatter.printHelp(getClass().getSimpleName(), options);
1565        System.exit(-1);
1566      }
1567
1568      Connection connection = ConnectionFactory.createConnection(getConf());
1569      Table table = connection.getTable(getTableName(getConf()));
1570
1571      Scan scan = new Scan();
1572      scan.setBatch(10000);
1573
1574      if (cmd.hasOption("s"))
1575        scan.setStartRow(Bytes.toBytesBinary(cmd.getOptionValue("s")));
1576
1577      if (cmd.hasOption("e"))
1578        scan.setStopRow(Bytes.toBytesBinary(cmd.getOptionValue("e")));
1579
1580      int limit = 0;
1581      if (cmd.hasOption("l"))
1582        limit = Integer.parseInt(cmd.getOptionValue("l"));
1583      else
1584        limit = 100;
1585
1586      ResultScanner scanner = table.getScanner(scan);
1587
1588      CINode node = new CINode();
1589      Result result = scanner.next();
1590      int count = 0;
1591      while (result != null && count++ < limit) {
1592        node = getCINode(result, node);
1593        System.out.printf("%s:%s:%012d:%s\n", Bytes.toStringBinary(node.key),
1594            Bytes.toStringBinary(node.prev), node.count, node.client);
1595        result = scanner.next();
1596      }
1597      scanner.close();
1598      table.close();
1599      connection.close();
1600
1601      return 0;
1602    }
1603  }
1604
1605  /**
1606   * A stand alone program that deletes a single node.
1607   */
1608  private static class Delete extends Configured implements Tool {
1609    @Override
1610    public int run(String[] args) throws Exception {
1611      if (args.length != 1) {
1612        System.out.println("Usage : " + Delete.class.getSimpleName() + " <node to delete>");
1613        return 0;
1614      }
1615      byte[] val = Bytes.toBytesBinary(args[0]);
1616
1617      org.apache.hadoop.hbase.client.Delete delete
1618        = new org.apache.hadoop.hbase.client.Delete(val);
1619
1620      try (Connection connection = ConnectionFactory.createConnection(getConf());
1621          Table table = connection.getTable(getTableName(getConf()))) {
1622        table.delete(delete);
1623      }
1624
1625      System.out.println("Delete successful");
1626      return 0;
1627    }
1628  }
1629
1630  abstract static class WalkerBase extends Configured{
1631    protected static CINode findStartNode(Table table, byte[] startKey) throws IOException {
1632      Scan scan = new Scan();
1633      scan.setStartRow(startKey);
1634      scan.setBatch(1);
1635      scan.addColumn(FAMILY_NAME, COLUMN_PREV);
1636
1637      long t1 = System.currentTimeMillis();
1638      ResultScanner scanner = table.getScanner(scan);
1639      Result result = scanner.next();
1640      long t2 = System.currentTimeMillis();
1641      scanner.close();
1642
1643      if ( result != null) {
1644        CINode node = getCINode(result, new CINode());
1645        System.out.printf("FSR %d %s\n", t2 - t1, Bytes.toStringBinary(node.key));
1646        return node;
1647      }
1648
1649      System.out.println("FSR " + (t2 - t1));
1650
1651      return null;
1652    }
1653    protected CINode getNode(byte[] row, Table table, CINode node) throws IOException {
1654      Get get = new Get(row);
1655      get.addColumn(FAMILY_NAME, COLUMN_PREV);
1656      Result result = table.get(get);
1657      return getCINode(result, node);
1658    }
1659  }
1660  /**
1661   * A stand alone program that follows a linked list created by {@link Generator} and prints
1662   * timing info.
1663   */
1664  private static class Walker extends WalkerBase implements Tool {
1665
1666    public Walker(){}
1667
1668    @Override
1669    public int run(String[] args) throws IOException {
1670
1671      Options options = new Options();
1672      options.addOption("n", "num", true, "number of queries");
1673      options.addOption("s", "start", true, "key to start at, binary string");
1674      options.addOption("l", "logevery", true, "log every N queries");
1675
1676      GnuParser parser = new GnuParser();
1677      CommandLine cmd = null;
1678      try {
1679        cmd = parser.parse(options, args);
1680        if (cmd.getArgs().length != 0) {
1681          throw new ParseException("Command takes no arguments");
1682        }
1683      } catch (ParseException e) {
1684        System.err.println("Failed to parse command line " + e.getMessage());
1685        System.err.println();
1686        HelpFormatter formatter = new HelpFormatter();
1687        formatter.printHelp(getClass().getSimpleName(), options);
1688        System.exit(-1);
1689      }
1690
1691      long maxQueries = Long.MAX_VALUE;
1692      if (cmd.hasOption('n')) {
1693        maxQueries = Long.parseLong(cmd.getOptionValue("n"));
1694      }
1695      Random rand = new SecureRandom();
1696      boolean isSpecificStart = cmd.hasOption('s');
1697
1698      byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
1699      int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1;
1700
1701      Connection connection = ConnectionFactory.createConnection(getConf());
1702      Table table = connection.getTable(getTableName(getConf()));
1703      long numQueries = 0;
1704      // If isSpecificStart is set, only walk one list from that particular node.
1705      // Note that in case of circular (or P-shaped) list it will walk forever, as is
1706      // the case in normal run without startKey.
1707      while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) {
1708        if (!isSpecificStart) {
1709          startKey = new byte[ROWKEY_LENGTH];
1710          rand.nextBytes(startKey);
1711        }
1712        CINode node = findStartNode(table, startKey);
1713        if (node == null && isSpecificStart) {
1714          System.err.printf("Start node not found: %s \n", Bytes.toStringBinary(startKey));
1715        }
1716        numQueries++;
1717        while (node != null && node.prev.length != NO_KEY.length &&
1718            numQueries < maxQueries) {
1719          byte[] prev = node.prev;
1720          long t1 = System.currentTimeMillis();
1721          node = getNode(prev, table, node);
1722          long t2 = System.currentTimeMillis();
1723          if (logEvery > 0 && numQueries % logEvery == 0) {
1724            System.out.printf("CQ %d: %d %s \n", numQueries, t2 - t1, Bytes.toStringBinary(prev));
1725          }
1726          numQueries++;
1727          if (node == null) {
1728            System.err.printf("UNDEFINED NODE %s \n", Bytes.toStringBinary(prev));
1729          } else if (node.prev.length == NO_KEY.length) {
1730            System.err.printf("TERMINATING NODE %s \n", Bytes.toStringBinary(node.key));
1731          }
1732        }
1733      }
1734      table.close();
1735      connection.close();
1736      return 0;
1737    }
1738  }
1739
1740  private static class Clean extends Configured implements Tool {
1741    @Override public int run(String[] args) throws Exception {
1742      if (args.length < 1) {
1743        System.err.println("Usage: Clean <output dir>");
1744        return -1;
1745      }
1746
1747      Path p = new Path(args[0]);
1748      Configuration conf = getConf();
1749      TableName tableName = getTableName(conf);
1750      try (FileSystem fs = HFileSystem.get(conf);
1751          Connection conn = ConnectionFactory.createConnection(conf);
1752          Admin admin = conn.getAdmin()) {
1753        if (admin.tableExists(tableName)) {
1754          admin.disableTable(tableName);
1755          admin.deleteTable(tableName);
1756        }
1757
1758        if (fs.exists(p)) {
1759          fs.delete(p, true);
1760        }
1761      }
1762
1763      return 0;
1764    }
1765  }
1766
1767  static TableName getTableName(Configuration conf) {
1768    return TableName.valueOf(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1769  }
1770
1771  private static CINode getCINode(Result result, CINode node) {
1772    node.key = Bytes.copy(result.getRow());
1773    if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
1774      node.prev = Bytes.copy(result.getValue(FAMILY_NAME, COLUMN_PREV));
1775    } else {
1776      node.prev = NO_KEY;
1777    }
1778    if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
1779      node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
1780    } else {
1781      node.count = -1;
1782    }
1783    if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
1784      node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
1785    } else {
1786      node.client = "";
1787    }
1788    return node;
1789  }
1790
1791  protected IntegrationTestingUtility util;
1792
1793  @Override
1794  public void setUpCluster() throws Exception {
1795    util = getTestingUtil(getConf());
1796    boolean isDistributed = util.isDistributedCluster();
1797    util.initializeCluster(isDistributed ? 1 : this.NUM_SLAVES_BASE);
1798    if (!isDistributed) {
1799      util.startMiniMapReduceCluster();
1800    }
1801    this.setConf(util.getConfiguration());
1802  }
1803
1804  @Override
1805  public void cleanUpCluster() throws Exception {
1806    super.cleanUpCluster();
1807    if (util.isDistributedCluster()) {
1808      util.shutdownMiniMapReduceCluster();
1809    }
1810  }
1811
1812  private static boolean isMultiUnevenColumnFamilies(Configuration conf) {
1813    return conf.getBoolean(Generator.MULTIPLE_UNEVEN_COLUMNFAMILIES_KEY,true);
1814  }
1815
1816  @Test
1817  public void testContinuousIngest() throws IOException, Exception {
1818    //Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers>
1819    Configuration conf = getTestingUtil(getConf()).getConfiguration();
1820    if (isMultiUnevenColumnFamilies(getConf())) {
1821      // make sure per CF flush is on
1822      conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushAllLargeStoresPolicy.class.getName());
1823    }
1824    int ret =
1825        ToolRunner.run(conf, new Loop(), new String[] { "1", "1", "2000000",
1826            util.getDataTestDirOnTestFS("IntegrationTestBigLinkedList").toString(), "1" });
1827    org.junit.Assert.assertEquals(0, ret);
1828  }
1829
1830  private void usage() {
1831    System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1832    printCommands();
1833  }
1834
1835  private void printCommands() {
1836    System.err.println("Commands:");
1837    System.err.println(" generator  Map only job that generates data.");
1838    System.err.println(" verify     A map reduce job that looks for holes. Check return code and");
1839    System.err.println("            look at the counts after running. See REFERENCED and");
1840    System.err.println("            UNREFERENCED are ok. Any UNDEFINED counts are bad. Do not run");
1841    System.err.println("            with the Generator.");
1842    System.err.println(" walker     " +
1843      "Standalone program that starts following a linked list & emits timing info.");
1844    System.err.println(" print      Standalone program that prints nodes in the linked list.");
1845    System.err.println(" delete     Standalone program that deletes a·single node.");
1846    System.err.println(" loop       Program to Loop through Generator and Verify steps");
1847    System.err.println(" clean      Program to clean all left over detritus.");
1848    System.err.println(" search     Search for missing keys.");
1849    System.err.println("");
1850    System.err.println("General options:");
1851    System.err.println(" -D"+ TABLE_NAME_KEY+ "=<tableName>");
1852    System.err.println("    Run using the <tableName> as the tablename.  Defaults to "
1853        + DEFAULT_TABLE_NAME);
1854    System.err.println(" -D"+ HBaseTestingUtility.REGIONS_PER_SERVER_KEY+ "=<# regions>");
1855    System.err.println("    Create table with presplit regions per server.  Defaults to "
1856        + HBaseTestingUtility.DEFAULT_REGIONS_PER_SERVER);
1857
1858    System.err.println(" -DuseMob=<true|false>");
1859    System.err.println("    Create table so that the mob read/write path is forced.  " +
1860        "Defaults to false");
1861
1862    System.err.flush();
1863  }
1864
1865  @Override
1866  protected void processOptions(CommandLine cmd) {
1867    super.processOptions(cmd);
1868    String[] args = cmd.getArgs();
1869    //get the class, run with the conf
1870    if (args.length < 1) {
1871      printUsage(this.getClass().getSimpleName() +
1872        " <general options> COMMAND [<COMMAND options>]", "General options:", "");
1873      printCommands();
1874      // Have to throw an exception here to stop the processing. Looks ugly but gets message across.
1875      throw new RuntimeException("Incorrect Number of args.");
1876    }
1877    toRun = args[0];
1878    otherArgs = Arrays.copyOfRange(args, 1, args.length);
1879  }
1880
1881  @Override
1882  public int runTestFromCommandLine() throws Exception {
1883    Tool tool = null;
1884    if (toRun.equalsIgnoreCase("Generator")) {
1885      tool = new Generator();
1886    } else if (toRun.equalsIgnoreCase("Verify")) {
1887      tool = new Verify();
1888    } else if (toRun.equalsIgnoreCase("Loop")) {
1889      Loop loop = new Loop();
1890      loop.it = this;
1891      tool = loop;
1892    } else if (toRun.equalsIgnoreCase("Walker")) {
1893      tool = new Walker();
1894    } else if (toRun.equalsIgnoreCase("Print")) {
1895      tool = new Print();
1896    } else if (toRun.equalsIgnoreCase("Delete")) {
1897      tool = new Delete();
1898    } else if (toRun.equalsIgnoreCase("Clean")) {
1899      tool = new Clean();
1900    } else if (toRun.equalsIgnoreCase("Search")) {
1901      tool = new Search();
1902    } else {
1903      usage();
1904      throw new RuntimeException("Unknown arg");
1905    }
1906
1907    return ToolRunner.run(getConf(), tool, otherArgs);
1908  }
1909
1910  @Override
1911  public TableName getTablename() {
1912    Configuration c = getConf();
1913    return TableName.valueOf(c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1914  }
1915
1916  @Override
1917  protected Set<String> getColumnFamilies() {
1918    if (isMultiUnevenColumnFamilies(getConf())) {
1919      return Sets.newHashSet(Bytes.toString(FAMILY_NAME), Bytes.toString(BIG_FAMILY_NAME),
1920        Bytes.toString(TINY_FAMILY_NAME));
1921    } else {
1922      return Sets.newHashSet(Bytes.toString(FAMILY_NAME));
1923    }
1924  }
1925
1926  private static void setJobConf(Job job, int numMappers, long numNodes,
1927      Integer width, Integer wrapMultiplier, Integer numWalkers) {
1928    job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1929    job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1930    if (width != null) {
1931      job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width);
1932    }
1933    if (wrapMultiplier != null) {
1934      job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMultiplier);
1935    }
1936    if (numWalkers != null) {
1937      job.getConfiguration().setInt(CONCURRENT_WALKER_KEY, numWalkers);
1938    }
1939  }
1940
1941  public static void setJobScannerConf(Job job) {
1942    // Make sure scanners log something useful to make debugging possible.
1943    job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1944    job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1945  }
1946
1947  public static void main(String[] args) throws Exception {
1948    Configuration conf = HBaseConfiguration.create();
1949    IntegrationTestingUtility.setUseDistributedCluster(conf);
1950    int ret = ToolRunner.run(conf, new IntegrationTestBigLinkedList(), args);
1951    System.exit(ret);
1952  }
1953}