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