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