View Javadoc

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.util.ArrayList;
25  import java.util.Arrays;
26  import java.util.List;
27  import java.util.Random;
28  import java.util.Set;
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.Path;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HColumnDescriptor;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.IntegrationTestBase;
45  import org.apache.hadoop.hbase.IntegrationTestingUtility;
46  import org.apache.hadoop.hbase.IntegrationTests;
47  import org.apache.hadoop.hbase.TableName;
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 IntegrationTestBase {
153   private static final byte[] NO_KEY = new byte[1];
154 
155   protected static String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
156 
157   protected static 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   private static final int ROWKEY_LENGTH = 16;
188 
189   private String toRun;
190   private String[] otherArgs;
191 
192   static class CINode {
193     byte[] key;
194     byte[] prev;
195     String client;
196     long count;
197   }
198 
199   /**
200    * A Map only job that generates random linked list and stores them.
201    */
202   static class Generator extends Configured implements Tool {
203 
204     private static final Log LOG = LogFactory.getLog(Generator.class);
205 
206     public static enum Counts {
207       UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
208     }
209 
210     static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
211       static class GeneratorInputSplit extends InputSplit implements Writable {
212         @Override
213         public long getLength() throws IOException, InterruptedException {
214           return 1;
215         }
216         @Override
217         public String[] getLocations() throws IOException, InterruptedException {
218           return new String[0];
219         }
220         @Override
221         public void readFields(DataInput arg0) throws IOException {
222         }
223         @Override
224         public void write(DataOutput arg0) throws IOException {
225         }
226       }
227 
228       static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
229         private long count;
230         private long numNodes;
231         private Random rand;
232 
233         @Override
234         public void close() throws IOException {
235         }
236 
237         @Override
238         public BytesWritable getCurrentKey() throws IOException, InterruptedException {
239           byte[] bytes = new byte[ROWKEY_LENGTH];
240           rand.nextBytes(bytes);
241           return new BytesWritable(bytes);
242         }
243 
244         @Override
245         public NullWritable getCurrentValue() throws IOException, InterruptedException {
246           return NullWritable.get();
247         }
248 
249         @Override
250         public float getProgress() throws IOException, InterruptedException {
251           return (float)(count / (double)numNodes);
252         }
253 
254         @Override
255         public void initialize(InputSplit arg0, TaskAttemptContext context)
256             throws IOException, InterruptedException {
257           numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
258           rand = new Random();
259         }
260 
261         @Override
262         public boolean nextKeyValue() throws IOException, InterruptedException {
263           return count++ < numNodes;
264         }
265 
266       }
267 
268       @Override
269       public RecordReader<BytesWritable,NullWritable> createRecordReader(
270           InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
271         GeneratorRecordReader rr = new GeneratorRecordReader();
272         rr.initialize(split, context);
273         return rr;
274       }
275 
276       @Override
277       public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
278         int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
279 
280         ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
281 
282         for (int i = 0; i < numMappers; i++) {
283           splits.add(new GeneratorInputSplit());
284         }
285 
286         return splits;
287       }
288     }
289 
290     /** Ensure output files from prev-job go to map inputs for current job */
291     static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
292       @Override
293       protected boolean isSplitable(JobContext context, Path filename) {
294         return false;
295       }
296     }
297 
298     /**
299      * Some ASCII art time:
300      * [ . . . ] represents one batch of random longs of length WIDTH
301      *
302      *                _________________________
303      *               |                  ______ |
304      *               |                 |      ||
305      *             __+_________________+_____ ||
306      *             v v                 v     |||
307      * first   = [ . . . . . . . . . . . ]   |||
308      *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
309      *             | | | | | | | | | | |     |||
310      * prev    = [ . . . . . . . . . . . ]   |||
311      *             ^ ^ ^ ^ ^ ^ ^ ^ ^ ^ ^     |||
312      *             | | | | | | | | | | |     |||
313      * current = [ . . . . . . . . . . . ]   |||
314      *                                       |||
315      * ...                                   |||
316      *                                       |||
317      * last    = [ . . . . . . . . . . . ]   |||
318      *             | | | | | | | | | | |-----|||
319      *             |                 |--------||
320      *             |___________________________|
321      */
322     static class GeneratorMapper
323       extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
324       Random rand = new Random();
325 
326       byte[][] first = null;
327       byte[][] prev = null;
328       byte[][] current = null;
329       byte[] id;
330       long count = 0;
331       int i;
332       HTable table;
333       long numNodes;
334       long wrap;
335       int width;
336 
337       protected void setup(Context context) throws IOException, InterruptedException {
338         id = Bytes.toBytes(UUID.randomUUID().toString());
339         Configuration conf = context.getConfiguration();
340         table = new HTable(conf, getTableName(conf));
341         table.setAutoFlush(false);
342         table.setWriteBufferSize(4 * 1024 * 1024);
343         this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
344         current = new byte[this.width][];
345         int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
346         this.wrap = (long)wrapMultiplier * width;
347         this.numNodes = context.getConfiguration().getLong(
348             GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
349         if (this.numNodes < this.wrap) {
350           this.wrap = this.numNodes;
351         }
352       };
353 
354       protected void cleanup(Context context) throws IOException ,InterruptedException {
355         table.close();
356       };
357 
358       @Override
359       protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
360         current[i] = new byte[key.getLength()];
361         System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
362         if (++i == current.length) {
363           persist(output, count, prev, current, id);
364           i = 0;
365 
366           if (first == null)
367             first = current;
368           prev = current;
369           current = new byte[this.width][];
370 
371           count += current.length;
372           output.setStatus("Count " + count);
373 
374           if (count % wrap == 0) {
375             // this block of code turns the 1 million linked list of length 25 into one giant
376             //circular linked list of 25 million
377             circularLeftShift(first);
378 
379             persist(output, -1, prev, first, null);
380 
381             first = null;
382             prev = null;
383           }
384         }
385       }
386 
387       private static <T> void circularLeftShift(T[] first) {
388         T ez = first[0];
389         for (int i = 0; i < first.length - 1; i++)
390           first[i] = first[i + 1];
391         first[first.length - 1] = ez;
392       }
393 
394       private void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
395           throws IOException {
396         for (int i = 0; i < current.length; i++) {
397           Put put = new Put(current[i]);
398           put.add(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
399 
400           if (count >= 0) {
401             put.add(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
402           }
403           if (id != null) {
404             put.add(FAMILY_NAME, COLUMN_CLIENT, id);
405           }
406           table.put(put);
407 
408           if (i % 1000 == 0) {
409             // Tickle progress every so often else maprunner will think us hung
410             output.progress();
411           }
412         }
413 
414         table.flushCommits();
415       }
416     }
417 
418     @Override
419     public int run(String[] args) throws Exception {
420       if (args.length < 3) {
421         System.out.println("Usage : " + Generator.class.getSimpleName() +
422             " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>]");
423         System.out.println("   where <num nodes per map> should be a multiple of " +
424             " width*wrap multiplier, 25M by default");
425         return 0;
426       }
427 
428       int numMappers = Integer.parseInt(args[0]);
429       long numNodes = Long.parseLong(args[1]);
430       Path tmpOutput = new Path(args[2]);
431       Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
432       Integer wrapMuplitplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
433       return run(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
434     }
435 
436     protected void createSchema() throws IOException {
437       HBaseAdmin admin = new HBaseAdmin(getConf());
438       TableName tableName = getTableName(getConf());
439       if (!admin.tableExists(tableName)) {
440         HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
441         htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
442         admin.createTable(htd);
443       }
444       admin.close();
445     }
446 
447     public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
448         Integer width, Integer wrapMuplitplier) throws Exception {
449       LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
450           + ", numNodes=" + numNodes);
451       Job job = new Job(getConf());
452 
453       job.setJobName("Random Input Generator");
454       job.setNumReduceTasks(0);
455       job.setJarByClass(getClass());
456 
457       job.setInputFormatClass(GeneratorInputFormat.class);
458       job.setOutputKeyClass(BytesWritable.class);
459       job.setOutputValueClass(NullWritable.class);
460 
461       setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
462 
463       job.setMapperClass(Mapper.class); //identity mapper
464 
465       FileOutputFormat.setOutputPath(job, tmpOutput);
466       job.setOutputFormatClass(SequenceFileOutputFormat.class);
467 
468       boolean success = job.waitForCompletion(true);
469 
470       return success ? 0 : 1;
471     }
472 
473     public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
474         Integer width, Integer wrapMuplitplier) throws Exception {
475       LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
476       createSchema();
477 
478       Job job = new Job(getConf());
479 
480       job.setJobName("Link Generator");
481       job.setNumReduceTasks(0);
482       job.setJarByClass(getClass());
483 
484       FileInputFormat.setInputPaths(job, tmpOutput);
485       job.setInputFormatClass(OneFilePerMapperSFIF.class);
486       job.setOutputKeyClass(NullWritable.class);
487       job.setOutputValueClass(NullWritable.class);
488 
489       setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
490 
491       job.setMapperClass(GeneratorMapper.class);
492 
493       job.setOutputFormatClass(NullOutputFormat.class);
494 
495       job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
496       TableMapReduceUtil.addDependencyJars(job);
497       TableMapReduceUtil.initCredentials(job);
498 
499       boolean success = job.waitForCompletion(true);
500 
501       return success ? 0 : 1;
502     }
503 
504     public int run(int numMappers, long numNodes, Path tmpOutput,
505         Integer width, Integer wrapMuplitplier) throws Exception {
506       int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
507       if (ret > 0) {
508         return ret;
509       }
510       return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
511     }
512   }
513 
514   /**
515    * A Map Reduce job that verifies that the linked lists generated by
516    * {@link Generator} do not have any holes.
517    */
518   static class Verify extends Configured implements Tool {
519 
520     private static final Log LOG = LogFactory.getLog(Verify.class);
521     private static final BytesWritable DEF = new BytesWritable(NO_KEY);
522 
523     private Job job;
524 
525     public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
526       private BytesWritable row = new BytesWritable();
527       private BytesWritable ref = new BytesWritable();
528 
529       @Override
530       protected void map(ImmutableBytesWritable key, Result value, Context context)
531           throws IOException ,InterruptedException {
532         byte[] rowKey = key.get();
533         row.set(rowKey, 0, rowKey.length);
534         context.write(row, DEF);
535         byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
536         if (prev != null && prev.length > 0) {
537           ref.set(prev, 0, prev.length);
538           context.write(ref, row);
539         } else {
540           LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
541         }
542       }
543     }
544 
545     public static enum Counts {
546       UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES
547     }
548 
549     public static class VerifyReducer extends Reducer<BytesWritable,BytesWritable,Text,Text> {
550       private ArrayList<byte[]> refs = new ArrayList<byte[]>();
551 
552       public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
553           throws IOException, InterruptedException {
554 
555         int defCount = 0;
556 
557         refs.clear();
558         for (BytesWritable type : values) {
559           if (type.getLength() == DEF.getLength()) {
560             defCount++;
561           } else {
562             byte[] bytes = new byte[type.getLength()];
563             System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
564             refs.add(bytes);
565           }
566         }
567 
568         // TODO check for more than one def, should not happen
569 
570         StringBuilder refsSb = null;
571         String keyString = null;
572         if (defCount == 0 || refs.size() != 1) {
573           refsSb = new StringBuilder();
574           String comma = "";
575           for (byte[] ref : refs) {
576             refsSb.append(comma);
577             comma = ",";
578             refsSb.append(Bytes.toStringBinary(ref));
579           }
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()).getName(), 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   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   static TableName getTableName(Configuration conf) {
951     return TableName.valueOf(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
952   }
953 
954   private static CINode getCINode(Result result, CINode node) {
955     node.key = Bytes.copy(result.getRow());
956     if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
957       node.prev = Bytes.copy(result.getValue(FAMILY_NAME, COLUMN_PREV));
958     } else {
959       node.prev = NO_KEY;
960     }
961     if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
962       node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
963     } else {
964       node.count = -1;
965     }
966     if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
967       node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
968     } else {
969       node.client = "";
970     }
971     return node;
972   }
973 
974   protected IntegrationTestingUtility util;
975 
976   @Before
977   @Override
978   public void setUp() throws Exception {
979     util = getTestingUtil(getConf());
980     util.initializeCluster(this.NUM_SLAVES_BASE);
981     this.setConf(util.getConfiguration());
982   }
983 
984   @After
985   @Override
986   public void cleanUp() throws Exception {
987     util.restoreCluster();
988   }
989 
990   @Test
991   public void testContinuousIngest() throws IOException, Exception {
992     //Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers>
993     int ret = ToolRunner.run(getTestingUtil(getConf()).getConfiguration(), new Loop(),
994         new String[] {"1", "1", "2000000",
995                      util.getDataTestDirOnTestFS("IntegrationTestBigLinkedList").toString(), "1"});
996     org.junit.Assert.assertEquals(0, ret);
997   }
998 
999   private void usage() {
1000     System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1001     System.err.println("  where COMMAND is one of:");
1002     System.err.println("");
1003     System.err.println("  Generator                  A map only job that generates data.");
1004     System.err.println("  Verify                     A map reduce job that looks for holes");
1005     System.err.println("                             Look at the counts after running");
1006     System.err.println("                             REFERENCED and UNREFERENCED are ok");
1007     System.err.println("                             any UNDEFINED counts are bad. Do not");
1008     System.err.println("                             run at the same time as the Generator.");
1009     System.err.println("  Walker                     A standalong program that starts ");
1010     System.err.println("                             following a linked list and emits");
1011     System.err.println("                             timing info.");
1012     System.err.println("  Print                      A standalone program that prints nodes");
1013     System.err.println("                             in the linked list.");
1014     System.err.println("  Delete                     A standalone program that deletes a·");
1015     System.err.println("                             single node.");
1016     System.err.println("  Loop                       A program to Loop through Generator and");
1017     System.err.println("                             Verify steps");
1018     System.err.println("\t  ");
1019     System.err.flush();
1020   }
1021 
1022   @Override
1023   protected void processOptions(CommandLine cmd) {
1024     super.processOptions(cmd);
1025     String[] args = cmd.getArgs();
1026     //get the class, run with the conf
1027     if (args.length < 1) {
1028       printUsage();
1029       throw new RuntimeException("Incorrect Number of args.");
1030     }
1031     toRun = args[0];
1032     otherArgs = Arrays.copyOfRange(args, 1, args.length);
1033   }
1034 
1035   @Override
1036   public int runTestFromCommandLine() throws Exception {
1037 
1038     Tool tool = null;
1039     if (toRun.equals("Generator")) {
1040       tool = new Generator();
1041     } else if (toRun.equals("Verify")) {
1042       tool = new Verify();
1043     } else if (toRun.equals("Loop")) {
1044       tool = new Loop();
1045     } else if (toRun.equals("Walker")) {
1046       tool = new Walker();
1047     } else if (toRun.equals("Print")) {
1048       tool = new Print();
1049     } else if (toRun.equals("Delete")) {
1050       tool = new Delete();
1051     } else {
1052       usage();
1053       throw new RuntimeException("Unknown arg");
1054     }
1055 
1056     return ToolRunner.run(getConf(), tool, otherArgs);
1057   }
1058 
1059   @Override
1060   public String getTablename() {
1061     Configuration c = getConf();
1062     return c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME);
1063   }
1064 
1065   @Override
1066   protected Set<String> getColumnFamilies() {
1067     return null;
1068   }
1069 
1070   private static void setJobConf(Job job, int numMappers, long numNodes,
1071       Integer width, Integer wrapMuplitplier) {
1072     job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1073     job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1074     if (width != null) {
1075       job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width.intValue());
1076     }
1077     if (wrapMuplitplier != null) {
1078       job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMuplitplier.intValue());
1079     }
1080   }
1081 
1082   private static void setJobScannerConf(Job job) {
1083     // Make sure scanners log something useful to make debugging possible.
1084     job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1085     job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1086   }
1087 
1088   public static void main(String[] args) throws Exception {
1089     Configuration conf = HBaseConfiguration.create();
1090     IntegrationTestingUtility.setUseDistributedCluster(conf);
1091     int ret = ToolRunner.run(conf, new IntegrationTestBigLinkedList(), args);
1092     System.exit(ret);
1093   }
1094 }