1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.test;
20  
21  import java.io.DataInput;
22  import java.io.DataOutput;
23  import java.io.IOException;
24  import java.math.BigInteger;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.List;
28  import java.util.Random;
29  import java.util.UUID;
30  
31  import org.apache.commons.cli.CommandLine;
32  import org.apache.commons.cli.GnuParser;
33  import org.apache.commons.cli.HelpFormatter;
34  import org.apache.commons.cli.Options;
35  import org.apache.commons.cli.ParseException;
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.conf.Configured;
40  import org.apache.hadoop.fs.FileSystem;
41  import org.apache.hadoop.fs.Path;
42  import org.apache.hadoop.hbase.HColumnDescriptor;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.IntegrationTestingUtility;
45  import org.apache.hadoop.hbase.IntegrationTests;
46  import org.apache.hadoop.hbase.client.Get;
47  import org.apache.hadoop.hbase.client.HBaseAdmin;
48  import org.apache.hadoop.hbase.client.HTable;
49  import org.apache.hadoop.hbase.client.Put;
50  import org.apache.hadoop.hbase.client.Result;
51  import org.apache.hadoop.hbase.client.ResultScanner;
52  import org.apache.hadoop.hbase.client.Scan;
53  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
54  import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
55  import org.apache.hadoop.hbase.mapreduce.TableMapper;
56  import org.apache.hadoop.hbase.util.Bytes;
57  import org.apache.hadoop.io.LongWritable;
58  import org.apache.hadoop.io.NullWritable;
59  import org.apache.hadoop.io.Text;
60  import org.apache.hadoop.io.VLongWritable;
61  import org.apache.hadoop.io.Writable;
62  import org.apache.hadoop.mapreduce.Counter;
63  import org.apache.hadoop.mapreduce.Counters;
64  import org.apache.hadoop.mapreduce.InputFormat;
65  import org.apache.hadoop.mapreduce.InputSplit;
66  import org.apache.hadoop.mapreduce.Job;
67  import org.apache.hadoop.mapreduce.JobContext;
68  import org.apache.hadoop.mapreduce.Mapper;
69  import org.apache.hadoop.mapreduce.RecordReader;
70  import org.apache.hadoop.mapreduce.Reducer;
71  import org.apache.hadoop.mapreduce.TaskAttemptContext;
72  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
73  import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
74  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
75  import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
76  import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
77  import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
78  import org.apache.hadoop.util.Tool;
79  import org.apache.hadoop.util.ToolRunner;
80  import org.junit.After;
81  import org.junit.Before;
82  import org.junit.Test;
83  import org.junit.experimental.categories.Category;
84  
85  /**
86   * This is an integration test borrowed from goraci, written by Keith Turner,
87   * which is in turn inspired by the Accumulo test called continous ingest (ci).
88   * The original source code can be found here:
89   * https://github.com/keith-turner/goraci
90   * https://github.com/enis/goraci/
91   *
92   * Apache Accumulo [0] has a simple test suite that verifies that data is not
93   * lost at scale. This test suite is called continuous ingest. This test runs
94   * many ingest clients that continually create linked lists containing 25
95   * million nodes. At some point the clients are stopped and a map reduce job is
96   * run to ensure no linked list has a hole. A hole indicates data was lost.··
97   *
98   * The nodes in the linked list are random. This causes each linked list to
99   * spread across the table. Therefore if one part of a table loses data, then it
100  * will be detected by references in another part of the table.
101  *
102  * THE ANATOMY OF THE TEST
103  *
104  * Below is rough sketch of how data is written. For specific details look at
105  * the Generator code.
106  *
107  * 1 Write out 1 million nodes· 2 Flush the client· 3 Write out 1 million that
108  * reference previous million· 4 If this is the 25th set of 1 million nodes,
109  * then update 1st set of million to point to last· 5 goto 1
110  *
111  * The key is that nodes only reference flushed nodes. Therefore a node should
112  * never reference a missing node, even if the ingest client is killed at any
113  * point in time.
114  *
115  * When running this test suite w/ Accumulo there is a script running in
116  * parallel called the Aggitator that randomly and continuously kills server
117  * processes.·· The outcome was that many data loss bugs were found in Accumulo
118  * by doing this.· This test suite can also help find bugs that impact uptime
119  * and stability when· run for days or weeks.··
120  *
121  * This test suite consists the following· - a few Java programs· - a little
122  * helper script to run the java programs - a maven script to build it.··
123  *
124  * When generating data, its best to have each map task generate a multiple of
125  * 25 million. The reason for this is that circular linked list are generated
126  * every 25M. Not generating a multiple in 25M will result in some nodes in the
127  * linked list not having references. The loss of an unreferenced node can not
128  * be detected.
129  *
130  *
131  * Below is a description of the Java programs
132  *
133  * Generator - A map only job that generates data. As stated previously,·
134  * its best to generate data in multiples of 25M.
135  *
136  * Verify - A map reduce job that looks for holes. Look at the counts after running. REFERENCED and
137  * UNREFERENCED are· ok, any UNDEFINED counts are bad. Do not run at the· same
138  * time as the Generator.
139  *
140  * Walker - A standalong program that start following a linked list· and emits timing info.··
141  *
142  * Print - A standalone program that prints nodes in the linked list
143  *
144  * Delete - A standalone program that deletes a single node
145  *
146  * This class can be run as a unit test, as an integration test, or from the command line
147  */
148 @Category(IntegrationTests.class)
149 public class IntegrationTestBigLinkedList extends Configured implements Tool {
150 
151   private static final String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
152 
153   private static final String DEFAULT_TABLE_NAME = "ci";
154 
155   private static byte[] FAMILY_NAME = Bytes.toBytes("meta");
156 
157   //link to the id of the prev node in the linked list
158   private static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
159 
160   //identifier of the mapred task that generated this row
161   private static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
162 
163   //the id of the row within the same client.
164   private static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
165 
166   /** How many rows to write per map task. This has to be a multiple of 25M */
167   private static final String GENERATOR_NUM_ROWS_PER_MAP_KEY
168     = "IntegrationTestBigLinkedList.generator.num_rows";
169 
170   private static final String GENERATOR_NUM_MAPPERS_KEY
171     = "IntegrationTestBigLinkedList.generator.map.tasks";
172 
173   static class CINode {
174     long key;
175     long prev;
176     String client;
177     long count;
178   }
179 
180   /**
181    * A Map only job that generates random linked list and stores them.
182    */
183   static class Generator extends Configured implements Tool {
184 
185     private static final Log LOG = LogFactory.getLog(Generator.class);
186 
187     private static final int WIDTH = 1000000;
188     private static final int WRAP = WIDTH * 25;
189 
190     public static enum Counts {
191       UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
192     }
193 
194     static class GeneratorInputFormat extends InputFormat<LongWritable,NullWritable> {
195       static class GeneratorInputSplit extends InputSplit implements Writable {
196         @Override
197         public long getLength() throws IOException, InterruptedException {
198           return 1;
199         }
200         @Override
201         public String[] getLocations() throws IOException, InterruptedException {
202           return new String[0];
203         }
204         @Override
205         public void readFields(DataInput arg0) throws IOException {
206         }
207         @Override
208         public void write(DataOutput arg0) throws IOException {
209         }
210       }
211 
212       static class GeneratorRecordReader extends RecordReader<LongWritable,NullWritable> {
213         private long count;
214         private long numNodes;
215         private Random rand;
216 
217         @Override
218         public void close() throws IOException {
219         }
220 
221         @Override
222         public LongWritable getCurrentKey() throws IOException, InterruptedException {
223           return new LongWritable(Math.abs(rand.nextLong()));
224         }
225 
226         @Override
227         public NullWritable getCurrentValue() throws IOException, InterruptedException {
228           return NullWritable.get();
229         }
230 
231         @Override
232         public float getProgress() throws IOException, InterruptedException {
233           return (float)(count / (double)numNodes);
234         }
235 
236         @Override
237         public void initialize(InputSplit arg0, TaskAttemptContext context)
238             throws IOException, InterruptedException {
239           numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
240           rand = new Random();
241         }
242 
243         @Override
244         public boolean nextKeyValue() throws IOException, InterruptedException {
245           return count++ < numNodes;
246         }
247 
248       }
249 
250       @Override
251       public RecordReader<LongWritable,NullWritable> createRecordReader(
252           InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
253         GeneratorRecordReader rr = new GeneratorRecordReader();
254         rr.initialize(split, context);
255         return rr;
256       }
257 
258       @Override
259       public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
260         int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
261 
262         ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
263 
264         for (int i = 0; i < numMappers; i++) {
265           splits.add(new GeneratorInputSplit());
266         }
267 
268         return splits;
269       }
270     }
271 
272     /** Ensure output files from prev-job go to map inputs for current job */
273     static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
274       @Override
275       protected boolean isSplitable(JobContext context, Path filename) {
276         return false;
277       }
278     }
279 
280     /**
281      * Some ASCII art time:
282      * [ . . . ] represents one batch of random longs of length WIDTH
283      *
284      *                _________________________
285      *               |                  ______ |
286      *               |                 |      ||
287      *             __+_________________+_____ ||
288      *             v v                 v     |||
289      * first   = [ . . . . . . . . . . . ]   |||
290      *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
291      *             | | | | | | | | | | |     |||
292      * prev    = [ . . . . . . . . . . . ]   |||
293      *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
294      *             | | | | | | | | | | |     |||
295      * current = [ . . . . . . . . . . . ]   |||
296      *                                       |||
297      * ...                                   |||
298      *                                       |||
299      * last    = [ . . . . . . . . . . . ]   |||
300      *             | | | | | | | | | | |-----|||
301      *             |                 |--------||
302      *             |___________________________|
303      */
304     static class GeneratorMapper
305       extends Mapper<LongWritable, NullWritable, NullWritable, NullWritable> {
306       Random rand = new Random();
307 
308       long[] first = null;
309       long[] prev = null;
310       long[] current = new long[WIDTH];
311       byte[] id;
312       long count = 0;
313       int i;
314       HTable table;
315       long numNodes;
316       long wrap = WRAP;
317 
318       protected void setup(Context context) throws IOException, InterruptedException {
319         id = Bytes.toBytes(UUID.randomUUID().toString());
320         Configuration conf = context.getConfiguration();
321         table = new HTable(conf, getTableName(conf));
322         table.setAutoFlush(false);
323         table.setWriteBufferSize(4 * 1024 * 1024);
324         numNodes = context.getConfiguration().getLong(GENERATOR_NUM_MAPPERS_KEY, 25000000);
325         if (numNodes < 25000000) {
326           wrap = numNodes;
327         }
328       };
329 
330       protected void cleanup(Context context) throws IOException ,InterruptedException {
331         table.close();
332       };
333 
334       @Override
335       protected void map(LongWritable key, NullWritable value, Context output) throws IOException {
336         current[i++] = Math.abs(key.get());
337 
338         if (i == current.length) {
339           persist(output, count, prev, current, id);
340           i = 0;
341 
342           if (first == null)
343             first = current;
344           prev = current;
345           current = new long[WIDTH];
346 
347           count += current.length;
348           output.setStatus("Count " + count);
349 
350           if (count % wrap == 0) {
351             // this block of code turns the 1 million linked list of length 25 into one giant
352             //circular linked list of 25 million
353             circularLeftShift(first);
354 
355             persist(output, -1, prev, first, null);
356 
357             first = null;
358             prev = null;
359           }
360         }
361       }
362 
363       private static void circularLeftShift(long[] first) {
364         long ez = first[0];
365         for (int i = 0; i < first.length - 1; i++)
366           first[i] = first[i + 1];
367         first[first.length - 1] = ez;
368       }
369 
370       private void persist(Context output, long count, long[] prev, long[] current, byte[] id)
371           throws IOException {
372         for (int i = 0; i < current.length; i++) {
373           Put put = new Put(Bytes.toBytes(current[i]));
374           put.add(FAMILY_NAME, COLUMN_PREV, Bytes.toBytes(prev == null ? -1 : prev[i]));
375 
376           if (count > 0) {
377             put.add(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + 1));
378           }
379           if (id != null) {
380             put.add(FAMILY_NAME, COLUMN_CLIENT, id);
381           }
382           table.put(put);
383 
384           if (i % 1000 == 0) {
385             // Tickle progress every so often else maprunner will think us hung
386             output.progress();
387           }
388         }
389 
390         table.flushCommits();
391       }
392     }
393 
394     @Override
395     public int run(String[] args) throws Exception {
396       if (args.length < 3) {
397         System.out.println("Usage : " + Generator.class.getSimpleName() +
398             " <num mappers> <num nodes per map> <tmp output dir>");
399         System.out.println("   where <num nodes per map> should be a multiple of 25M");
400         return 0;
401       }
402 
403       int numMappers = Integer.parseInt(args[0]);
404       long numNodes = Long.parseLong(args[1]);
405       Path tmpOutput = new Path(args[2]);
406       return run(numMappers, numNodes, tmpOutput);
407     }
408 
409     protected void createSchema() throws IOException {
410       HBaseAdmin admin = new HBaseAdmin(getConf());
411       byte[] tableName = getTableName(getConf());
412       if (!admin.tableExists(tableName)) {
413         HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
414         htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
415         admin.createTable(htd);
416       }
417       admin.close();
418     }
419 
420     public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput)
421         throws Exception {
422       LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
423           + ", numNodes=" + numNodes);
424       Job job = new Job(getConf());
425 
426       job.setJobName("Random Input Generator");
427       job.setNumReduceTasks(0);
428       job.setJarByClass(getClass());
429 
430       job.setInputFormatClass(GeneratorInputFormat.class);
431       job.setOutputKeyClass(LongWritable.class);
432       job.setOutputValueClass(NullWritable.class);
433 
434       job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
435       job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
436 
437       job.setMapperClass(Mapper.class); //identity mapper
438 
439       FileOutputFormat.setOutputPath(job, tmpOutput);
440       job.setOutputFormatClass(SequenceFileOutputFormat.class);
441 
442       boolean success = job.waitForCompletion(true);
443 
444       return success ? 0 : 1;
445     }
446 
447     public int runGenerator(int numMappers, long numNodes, Path tmpOutput) throws Exception {
448       LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
449       createSchema();
450 
451       Job job = new Job(getConf());
452 
453       job.setJobName("Link Generator");
454       job.setNumReduceTasks(0);
455       job.setJarByClass(getClass());
456 
457       FileInputFormat.setInputPaths(job, tmpOutput);
458       job.setInputFormatClass(OneFilePerMapperSFIF.class);
459       job.setOutputKeyClass(NullWritable.class);
460       job.setOutputValueClass(NullWritable.class);
461 
462       job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
463       job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
464 
465       job.setMapperClass(GeneratorMapper.class);
466 
467       job.setOutputFormatClass(NullOutputFormat.class);
468 
469       job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
470 
471       boolean success = job.waitForCompletion(true);
472 
473       return success ? 0 : 1;
474     }
475 
476     public int run(int numMappers, long numNodes, Path tmpOutput) throws Exception {
477       int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput);
478       if (ret > 0) {
479         return ret;
480       }
481 
482       return runGenerator(numMappers, numNodes, tmpOutput);
483     }
484   }
485 
486   /**
487    * A Map Reduce job that verifies that the linked lists generated by
488    * {@link Generator} do not have any holes.
489    */
490   static class Verify extends Configured implements Tool {
491 
492     private static final Log LOG = LogFactory.getLog(Verify.class);
493     private static final VLongWritable DEF = new VLongWritable(-1);
494 
495     private Job job;
496 
497     public static class VerifyMapper extends TableMapper<LongWritable, VLongWritable> {
498       private LongWritable row = new LongWritable();
499       private LongWritable ref = new LongWritable();
500       private VLongWritable vrow = new VLongWritable();
501 
502       @Override
503       protected void map(ImmutableBytesWritable key, Result value, Context context)
504           throws IOException ,InterruptedException {
505         row.set(Bytes.toLong(key.get()));
506         context.write(row, DEF);
507 
508         long prev = Bytes.toLong(value.getValue(FAMILY_NAME, COLUMN_PREV));
509         if (prev >= 0) {
510           ref.set(prev);
511           vrow.set(Bytes.toLong(key.get()));
512           context.write(ref, vrow);
513         }
514       }
515     }
516 
517     public static enum Counts {
518       UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
519     }
520 
521     public static class VerifyReducer extends Reducer<LongWritable,VLongWritable,Text,Text> {
522       private ArrayList<Long> refs = new ArrayList<Long>();
523 
524       public void reduce(LongWritable key, Iterable<VLongWritable> values, Context context)
525           throws IOException, InterruptedException {
526 
527         int defCount = 0;
528 
529         refs.clear();
530         for (VLongWritable type : values) {
531           if (type.get() == -1) {
532             defCount++;
533           } else {
534             refs.add(type.get());
535           }
536         }
537 
538         // TODO check for more than one def, should not happen
539 
540         if (defCount == 0 && refs.size() > 0) {
541           // this is bad, found a node that is referenced but not defined. It must have been
542           //lost, emit some info about this node for debugging purposes.
543 
544           StringBuilder sb = new StringBuilder();
545           String comma = "";
546           for (Long ref : refs) {
547             sb.append(comma);
548             comma = ",";
549             sb.append(String.format("%016x", ref));
550           }
551 
552           context.write(new Text(String.format("%016x", key.get())), new Text(sb.toString()));
553           context.getCounter(Counts.UNDEFINED).increment(1);
554 
555         } else if (defCount > 0 && refs.size() == 0) {
556           // node is defined but not referenced
557           context.getCounter(Counts.UNREFERENCED).increment(1);
558         } else {
559           // node is defined and referenced
560           context.getCounter(Counts.REFERENCED).increment(1);
561         }
562 
563       }
564     }
565 
566     @Override
567     public int run(String[] args) throws Exception {
568 
569       if (args.length != 2) {
570         System.out.println("Usage : " + Verify.class.getSimpleName() + " <output dir> <num reducers>");
571         return 0;
572       }
573 
574       String outputDir = args[0];
575       int numReducers = Integer.parseInt(args[1]);
576 
577        return run(outputDir, numReducers);
578     }
579 
580     public int run(String outputDir, int numReducers) throws Exception {
581       return run(new Path(outputDir), numReducers);
582     }
583 
584     public int run(Path outputDir, int numReducers) throws Exception {
585       LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
586 
587       job = new Job(getConf());
588 
589       job.setJobName("Link Verifier");
590       job.setNumReduceTasks(numReducers);
591       job.setJarByClass(getClass());
592 
593       Scan scan = new Scan();
594       scan.addColumn(FAMILY_NAME, COLUMN_PREV);
595       scan.setCaching(10000);
596       scan.setCacheBlocks(false);
597 
598       TableMapReduceUtil.initTableMapperJob(getTableName(getConf()), scan,
599           VerifyMapper.class, LongWritable.class, VLongWritable.class, job);
600 
601       job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
602 
603       job.setReducerClass(VerifyReducer.class);
604       job.setOutputFormatClass(TextOutputFormat.class);
605       TextOutputFormat.setOutputPath(job, outputDir);
606 
607       boolean success = job.waitForCompletion(true);
608 
609       return success ? 0 : 1;
610     }
611 
612     public boolean verify(long expectedReferenced) throws Exception {
613       if (job == null) {
614         throw new IllegalStateException("You should call run() first");
615       }
616 
617       Counters counters = job.getCounters();
618 
619       Counter referenced = counters.findCounter(Counts.REFERENCED);
620       Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
621       Counter undefined = counters.findCounter(Counts.UNDEFINED);
622 
623       boolean success = true;
624       //assert
625       if (expectedReferenced != referenced.getValue()) {
626         LOG.error("Expected referenced count does not match with actual referenced count. " +
627             "expected referenced=" + expectedReferenced + " ,actual=" + referenced.getValue());
628         success = false;
629       }
630 
631       if (unreferenced.getValue() > 0) {
632         LOG.error("Unreferenced nodes were not expected. Unreferenced count=" + unreferenced.getValue());
633         success = false;
634       }
635 
636       if (undefined.getValue() > 0) {
637         LOG.error("Found an undefined node. Undefined count=" + undefined.getValue());
638         success = false;
639       }
640 
641       return success;
642     }
643   }
644 
645   /**
646    * Executes Generate and Verify in a loop. Data is not cleaned between runs, so each iteration
647    * adds more data.
648    */
649   private static class Loop extends Configured implements Tool {
650 
651     private static final Log LOG = LogFactory.getLog(Loop.class);
652 
653     protected void runGenerator(int numMappers, long numNodes, String outputDir) throws Exception {
654       Path outputPath = new Path(outputDir);
655       UUID uuid = UUID.randomUUID(); //create a random UUID.
656       Path generatorOutput = new Path(outputPath, uuid.toString());
657 
658       Generator generator = new Generator();
659       generator.setConf(getConf());
660       int retCode = generator.run(numMappers, numNodes, generatorOutput);
661 
662       if (retCode > 0) {
663         throw new RuntimeException("Generator failed with return code: " + retCode);
664       }
665     }
666 
667     protected void runVerify(String outputDir, int numReducers, long expectedNumNodes) throws Exception {
668       Path outputPath = new Path(outputDir);
669       UUID uuid = UUID.randomUUID(); //create a random UUID.
670       Path iterationOutput = new Path(outputPath, uuid.toString());
671 
672       Verify verify = new Verify();
673       verify.setConf(getConf());
674       int retCode = verify.run(iterationOutput, numReducers);
675       if (retCode > 0) {
676         throw new RuntimeException("Verify.run failed with return code: " + retCode);
677       }
678 
679       boolean verifySuccess = verify.verify(expectedNumNodes);
680       if (!verifySuccess) {
681         throw new RuntimeException("Verify.verify failed");
682       }
683 
684       LOG.info("Verify finished with succees. Total nodes=" + expectedNumNodes);
685     }
686 
687     @Override
688     public int run(String[] args) throws Exception {
689       if (args.length < 5) {
690         System.err.println("Usage: Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers>");
691         return 1;
692       }
693 
694       LOG.info("Running Loop with args:" + Arrays.deepToString(args));
695 
696       int numIterations = Integer.parseInt(args[0]);
697       int numMappers = Integer.parseInt(args[1]);
698       long numNodes = Long.parseLong(args[2]);
699       String outputDir = args[3];
700       int numReducers = Integer.parseInt(args[4]);
701 
702       long expectedNumNodes = 0;
703 
704       if (numIterations < 0) {
705         numIterations = Integer.MAX_VALUE; //run indefinitely (kind of)
706       }
707 
708       for (int i=0; i < numIterations; i++) {
709         LOG.info("Starting iteration = " + i);
710         runGenerator(numMappers, numNodes, outputDir);
711         expectedNumNodes += numMappers * numNodes;
712 
713         runVerify(outputDir, numReducers, expectedNumNodes);
714       }
715 
716       return 0;
717     }
718   }
719 
720   /**
721    * A stand alone program that prints out portions of a list created by {@link Generator}
722    */
723   private static class Print extends Configured implements Tool {
724     public int run(String[] args) throws Exception {
725       Options options = new Options();
726       options.addOption("s", "start", true, "start key");
727       options.addOption("e", "end", true, "end key");
728       options.addOption("l", "limit", true, "number to print");
729 
730       GnuParser parser = new GnuParser();
731       CommandLine cmd = null;
732       try {
733         cmd = parser.parse(options, args);
734         if (cmd.getArgs().length != 0) {
735           throw new ParseException("Command takes no arguments");
736         }
737       } catch (ParseException e) {
738         System.err.println("Failed to parse command line " + e.getMessage());
739         System.err.println();
740         HelpFormatter formatter = new HelpFormatter();
741         formatter.printHelp(getClass().getSimpleName(), options);
742         System.exit(-1);
743       }
744 
745       HTable table = new HTable(getConf(), getTableName(getConf()));
746 
747       Scan scan = new Scan();
748       scan.setBatch(10000);
749 
750       if (cmd.hasOption("s"))
751         scan.setStartRow(Bytes.toBytes(new BigInteger(cmd.getOptionValue("s"), 16).longValue()));
752 
753       if (cmd.hasOption("e"))
754         scan.setStopRow(Bytes.toBytes(new BigInteger(cmd.getOptionValue("e"), 16).longValue()));
755 
756       int limit = 0;
757       if (cmd.hasOption("l"))
758         limit = Integer.parseInt(cmd.getOptionValue("l"));
759       else
760         limit = 100;
761 
762       ResultScanner scanner = table.getScanner(scan);
763 
764       CINode node = new CINode();
765       Result result = scanner.next();
766       int count = 0;
767       while (result != null && count++ < limit) {
768         node = getCINode(result, node);
769         System.out.printf("%016x:%016x:%012d:%s\n", node.key, node.prev, node.count, node.client);
770         result = scanner.next();
771       }
772       scanner.close();
773       table.close();
774 
775       return 0;
776     }
777   }
778 
779   /**
780    * A stand alone program that deletes a single node.
781    */
782   private static class Delete extends Configured implements Tool {
783     public int run(String[] args) throws Exception {
784       if (args.length != 1) {
785         System.out.println("Usage : " + Delete.class.getSimpleName() + " <node to delete>");
786         return 0;
787       }
788       long val = new BigInteger(args[0], 16).longValue();
789 
790       org.apache.hadoop.hbase.client.Delete delete
791         = new org.apache.hadoop.hbase.client.Delete(Bytes.toBytes(val));
792 
793       HTable table = new HTable(getConf(), getTableName(getConf()));
794 
795       table.delete(delete);
796       table.flushCommits();
797       table.close();
798 
799       System.out.println("Delete successful");
800       return 0;
801     }
802   }
803 
804   /**
805    * A stand alone program that follows a linked list created by {@link Generator} and prints timing info.
806    */
807   private static class Walker extends Configured implements Tool {
808     public int run(String[] args) throws IOException {
809       Options options = new Options();
810       options.addOption("n", "num", true, "number of queries");
811 
812       GnuParser parser = new GnuParser();
813       CommandLine cmd = null;
814       try {
815         cmd = parser.parse(options, args);
816         if (cmd.getArgs().length != 0) {
817           throw new ParseException("Command takes no arguments");
818         }
819       } catch (ParseException e) {
820         System.err.println("Failed to parse command line " + e.getMessage());
821         System.err.println();
822         HelpFormatter formatter = new HelpFormatter();
823         formatter.printHelp(getClass().getSimpleName(), options);
824         System.exit(-1);
825       }
826 
827       long maxQueries = Long.MAX_VALUE;
828       if (cmd.hasOption('n')) {
829         maxQueries = Long.parseLong(cmd.getOptionValue("n"));
830       }
831 
832       HTable table = new HTable(getConf(), getTableName(getConf()));
833 
834       Random rand = new Random();
835 
836       long numQueries = 0;
837 
838       while (numQueries < maxQueries) {
839         CINode node = findStartNode(rand, table);
840         numQueries++;
841         while (node != null && node.prev >= 0 && numQueries < maxQueries) {
842           long prev = node.prev;
843 
844           long t1 = System.currentTimeMillis();
845           node = getNode(prev, table, node);
846           long t2 = System.currentTimeMillis();
847           System.out.printf("CQ %d %016x \n", t2 - t1, prev); //cold cache
848           numQueries++;
849 
850           t1 = System.currentTimeMillis();
851           node = getNode(prev, table, node);
852           t2 = System.currentTimeMillis();
853           System.out.printf("HQ %d %016x \n", t2 - t1, prev); //hot cache
854           numQueries++;
855         }
856       }
857 
858       table.close();
859       return 0;
860     }
861 
862     private static CINode findStartNode(Random rand, HTable table) throws IOException {
863       Scan scan = new Scan();
864       scan.setStartRow(Bytes.toBytes(Math.abs(rand.nextLong())));
865       scan.setBatch(1);
866       scan.addColumn(FAMILY_NAME, COLUMN_PREV);
867 
868       long t1 = System.currentTimeMillis();
869       ResultScanner scanner = table.getScanner(scan);
870       Result result = scanner.next();
871       long t2 = System.currentTimeMillis();
872       scanner.close();
873 
874       if ( result != null) {
875         CINode node = getCINode(result, new CINode());
876         System.out.printf("FSR %d %016x\n", t2 - t1, node.key);
877         return node;
878       }
879 
880       System.out.println("FSR " + (t2 - t1));
881 
882       return null;
883     }
884 
885     private CINode getNode(long row, HTable table, CINode node) throws IOException {
886       Get get = new Get(Bytes.toBytes(row));
887       get.addColumn(FAMILY_NAME, COLUMN_PREV);
888       Result result = table.get(get);
889       return getCINode(result, node);
890     }
891   }
892 
893   private static byte[] getTableName(Configuration conf) {
894     return Bytes.toBytes(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
895   }
896 
897   private static CINode getCINode(Result result, CINode node) {
898     node.key = Bytes.toLong(result.getRow());
899     if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
900       node.prev = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_PREV));
901     }
902     if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
903       node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
904     }
905     if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
906       node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
907     }
908     return node;
909   }
910 
911   private IntegrationTestingUtility util;
912 
913   @Before
914   public void setUp() throws Exception {
915     util = getTestingUtil();
916     util.initializeCluster(3);
917     this.setConf(util.getConfiguration());
918   }
919 
920   @After
921   public void tearDown() throws Exception {
922     util.restoreCluster();
923   }
924 
925   @Test
926   public void testContinuousIngest() throws IOException, Exception {
927     //Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers>
928     int ret = ToolRunner.run(getTestingUtil().getConfiguration(), new Loop(),
929         new String[] {"1", "1", "2000000",
930                      getTestDir("IntegrationTestBigLinkedList", "testContinuousIngest").toString(), "1"});
931     org.junit.Assert.assertEquals(0, ret);
932   }
933 
934   public Path getTestDir(String testName, String subdir) throws IOException {
935     //HBaseTestingUtility.getDataTestDirOnTestFs() has not been backported.
936     FileSystem fs = FileSystem.get(getConf());
937     Path base = new Path(fs.getWorkingDirectory(), "test-data");
938     String randomStr = UUID.randomUUID().toString();
939     Path testDir = new Path(base, randomStr);
940     fs.deleteOnExit(testDir);
941 
942     return new Path(new Path(testDir, testName), subdir);
943   }
944 
945   private IntegrationTestingUtility getTestingUtil() {
946     if (this.util == null) {
947       if (getConf() == null) {
948         this.util = new IntegrationTestingUtility();
949       } else {
950         this.util = new IntegrationTestingUtility(getConf());
951       }
952     }
953     return util;
954   }
955 
956   private int printUsage() {
957     System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
958     System.err.println("  where COMMAND is one of:");
959     System.err.println("");
960     System.err.println("  Generator                  A map only job that generates data.");
961     System.err.println("  Verify                     A map reduce job that looks for holes");
962     System.err.println("                             Look at the counts after running");
963     System.err.println("                             REFERENCED and UNREFERENCED are ok");
964     System.err.println("                             any UNDEFINED counts are bad. Do not");
965     System.err.println("                             run at the same time as the Generator.");
966     System.err.println("  Walker                     A standalong program that starts ");
967     System.err.println("                             following a linked list and emits");
968     System.err.println("                             timing info.");
969     System.err.println("  Print                      A standalone program that prints nodes");
970     System.err.println("                             in the linked list.");
971     System.err.println("  Delete                     A standalone program that deletes a·");
972     System.err.println("                             single node.");
973     System.err.println("  Loop                       A program to Loop through Generator and");
974     System.err.println("                             Verify steps");
975     System.err.println("\t  ");
976     return 1;
977   }
978 
979   @Override
980   public int run(String[] args) throws Exception {
981     //get the class, run with the conf
982     if (args.length < 1) {
983       return printUsage();
984     }
985     Tool tool = null;
986     if (args[0].equals("Generator")) {
987       tool = new Generator();
988     } else if (args[0].equals("Verify")) {
989       tool = new Verify();
990     } else if (args[0].equals("Loop")) {
991       tool = new Loop();
992     } else if (args[0].equals("Walker")) {
993       tool = new Walker();
994     } else if (args[0].equals("Print")) {
995       tool = new Print();
996     } else if (args[0].equals("Delete")) {
997       tool = new Delete();
998     } else {
999       return printUsage();
1000     }
1001 
1002     args = Arrays.copyOfRange(args, 1, args.length);
1003     return ToolRunner.run(getConf(), tool, args);
1004   }
1005 
1006   public static void main(String[] args) throws Exception {
1007     int ret = ToolRunner.run(new IntegrationTestBigLinkedList(), args);
1008     System.exit(ret);
1009   }
1010 }