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