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