1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
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
169 private static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
170
171
172 private static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
173
174
175 private static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
176
177
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;
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
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
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
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
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
382
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
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);
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
523
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
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
597
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
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
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
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
733
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();
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();
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;
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
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
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
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
934
935
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 util.initializeCluster(util.isDistributedCluster() ? 1 : this.NUM_SLAVES_BASE);
1028 this.setConf(util.getConfiguration());
1029 }
1030
1031 @Test
1032 public void testContinuousIngest() throws IOException, Exception {
1033
1034 int ret = ToolRunner.run(getTestingUtil(getConf()).getConfiguration(), new Loop(),
1035 new String[] {"1", "1", "2000000",
1036 util.getDataTestDirOnTestFS("IntegrationTestBigLinkedList").toString(), "1"});
1037 org.junit.Assert.assertEquals(0, ret);
1038 }
1039
1040 private void usage() {
1041 System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1042 System.err.println(" where COMMAND is one of:");
1043 System.err.println("");
1044 System.err.println(" Generator A map only job that generates data.");
1045 System.err.println(" Verify A map reduce job that looks for holes");
1046 System.err.println(" Look at the counts after running");
1047 System.err.println(" REFERENCED and UNREFERENCED are ok");
1048 System.err.println(" any UNDEFINED counts are bad. Do not");
1049 System.err.println(" run at the same time as the Generator.");
1050 System.err.println(" Walker A standalong program that starts ");
1051 System.err.println(" following a linked list and emits");
1052 System.err.println(" timing info.");
1053 System.err.println(" Print A standalone program that prints nodes");
1054 System.err.println(" in the linked list.");
1055 System.err.println(" Delete A standalone program that deletes a·");
1056 System.err.println(" single node.");
1057 System.err.println(" Loop A program to Loop through Generator and");
1058 System.err.println(" Verify steps");
1059 System.err.println("\t ");
1060 System.err.flush();
1061 }
1062
1063 @Override
1064 protected void processOptions(CommandLine cmd) {
1065 super.processOptions(cmd);
1066 String[] args = cmd.getArgs();
1067
1068 if (args.length < 1) {
1069 printUsage();
1070 throw new RuntimeException("Incorrect Number of args.");
1071 }
1072 toRun = args[0];
1073 otherArgs = Arrays.copyOfRange(args, 1, args.length);
1074 }
1075
1076 @Override
1077 public int runTestFromCommandLine() throws Exception {
1078
1079 Tool tool = null;
1080 if (toRun.equals("Generator")) {
1081 tool = new Generator();
1082 } else if (toRun.equals("Verify")) {
1083 tool = new Verify();
1084 } else if (toRun.equals("Loop")) {
1085 Loop loop = new Loop();
1086 loop.it = this;
1087 tool = loop;
1088 } else if (toRun.equals("Walker")) {
1089 tool = new Walker();
1090 } else if (toRun.equals("Print")) {
1091 tool = new Print();
1092 } else if (toRun.equals("Delete")) {
1093 tool = new Delete();
1094 } else {
1095 usage();
1096 throw new RuntimeException("Unknown arg");
1097 }
1098
1099 return ToolRunner.run(getConf(), tool, otherArgs);
1100 }
1101
1102 @Override
1103 public String getTablename() {
1104 Configuration c = getConf();
1105 return c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME);
1106 }
1107
1108 @Override
1109 protected Set<String> getColumnFamilies() {
1110 return Sets.newHashSet(Bytes.toString(FAMILY_NAME));
1111 }
1112
1113 private static void setJobConf(Job job, int numMappers, long numNodes,
1114 Integer width, Integer wrapMultiplier) {
1115 job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1116 job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1117 if (width != null) {
1118 job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width);
1119 }
1120 if (wrapMultiplier != null) {
1121 job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMultiplier);
1122 }
1123 }
1124
1125 private static void setJobScannerConf(Job job) {
1126
1127 job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1128 job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1129 }
1130
1131 public static void main(String[] args) throws Exception {
1132 Configuration conf = HBaseConfiguration.create();
1133 IntegrationTestingUtility.setUseDistributedCluster(conf);
1134 int ret = ToolRunner.run(conf, new IntegrationTestBigLinkedList(), args);
1135 System.exit(ret);
1136 }
1137 }