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.io.StringWriter;
25 import java.math.BigInteger;
26 import java.util.ArrayList;
27 import java.util.Arrays;
28 import java.util.List;
29 import java.util.Random;
30 import java.util.UUID;
31
32 import org.apache.commons.cli.CommandLine;
33 import org.apache.commons.cli.GnuParser;
34 import org.apache.commons.cli.HelpFormatter;
35 import org.apache.commons.cli.Options;
36 import org.apache.commons.cli.ParseException;
37 import org.apache.commons.logging.Log;
38 import org.apache.commons.logging.LogFactory;
39 import org.apache.hadoop.conf.Configuration;
40 import org.apache.hadoop.conf.Configured;
41 import org.apache.hadoop.fs.FileSystem;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.hbase.HBaseConfiguration;
44 import org.apache.hadoop.hbase.HColumnDescriptor;
45 import org.apache.hadoop.hbase.HTableDescriptor;
46 import org.apache.hadoop.hbase.IntegrationTestingUtility;
47 import org.apache.hadoop.hbase.IntegrationTests;
48 import org.apache.hadoop.hbase.client.Get;
49 import org.apache.hadoop.hbase.client.HBaseAdmin;
50 import org.apache.hadoop.hbase.client.HTable;
51 import org.apache.hadoop.hbase.client.Put;
52 import org.apache.hadoop.hbase.client.Result;
53 import org.apache.hadoop.hbase.client.ResultScanner;
54 import org.apache.hadoop.hbase.client.Scan;
55 import org.apache.hadoop.hbase.client.ScannerCallable;
56 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
57 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
58 import org.apache.hadoop.hbase.mapreduce.TableMapper;
59 import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
60 import org.apache.hadoop.hbase.util.Bytes;
61 import org.apache.hadoop.io.BytesWritable;
62 import org.apache.hadoop.io.NullWritable;
63 import org.apache.hadoop.io.Text;
64 import org.apache.hadoop.io.Writable;
65 import org.apache.hadoop.mapreduce.Counter;
66 import org.apache.hadoop.mapreduce.Counters;
67 import org.apache.hadoop.mapreduce.InputFormat;
68 import org.apache.hadoop.mapreduce.InputSplit;
69 import org.apache.hadoop.mapreduce.Job;
70 import org.apache.hadoop.mapreduce.JobContext;
71 import org.apache.hadoop.mapreduce.Mapper;
72 import org.apache.hadoop.mapreduce.RecordReader;
73 import org.apache.hadoop.mapreduce.Reducer;
74 import org.apache.hadoop.mapreduce.TaskAttemptContext;
75 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
76 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
77 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
78 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
79 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
80 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
81 import org.apache.hadoop.util.Tool;
82 import org.apache.hadoop.util.ToolRunner;
83 import org.junit.After;
84 import org.junit.Before;
85 import org.junit.Test;
86 import org.junit.experimental.categories.Category;
87
88
89
90
91
92
93
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 @Category(IntegrationTests.class)
152 public class IntegrationTestBigLinkedList extends Configured implements Tool {
153 private static final byte[] NO_KEY = new byte[1];
154
155 private static final String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
156
157 private static final String DEFAULT_TABLE_NAME = "IntegrationTestBigLinkedList";
158
159 private static byte[] FAMILY_NAME = Bytes.toBytes("meta");
160
161
162 private static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
163
164
165 private static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
166
167
168 private static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
169
170
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;
184
185 private static final int WIDTH_DEFAULT = 1000000;
186 private static final int WRAP_DEFAULT = 25;
187
188 private static final int ROWKEY_LENGTH = 16;
189
190 static class CINode {
191 byte[] key;
192 byte[] prev;
193
194 String client;
195 long count;
196 }
197
198
199
200
201 static class Generator extends Configured implements Tool {
202
203 private static final Log LOG = LogFactory.getLog(Generator.class);
204
205 public static enum Counts {
206 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
207 }
208
209 static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
210 static class GeneratorInputSplit extends InputSplit implements Writable {
211 @Override
212 public long getLength() throws IOException, InterruptedException {
213 return 1;
214 }
215 @Override
216 public String[] getLocations() throws IOException, InterruptedException {
217 return new String[0];
218 }
219 @Override
220 public void readFields(DataInput arg0) throws IOException {
221 }
222 @Override
223 public void write(DataOutput arg0) throws IOException {
224 }
225 }
226
227 static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
228 private long count;
229 private long numNodes;
230 private Random rand;
231
232 @Override
233 public void close() throws IOException {
234 }
235
236 @Override
237 public BytesWritable getCurrentKey() throws IOException, InterruptedException {
238 byte[] bytes = new byte[ROWKEY_LENGTH];
239 rand.nextBytes(bytes);
240 return new BytesWritable(bytes);
241 }
242
243 @Override
244 public NullWritable getCurrentValue() throws IOException, InterruptedException {
245 return NullWritable.get();
246 }
247
248 @Override
249 public float getProgress() throws IOException, InterruptedException {
250 return (float)(count / (double)numNodes);
251 }
252
253 @Override
254 public void initialize(InputSplit arg0, TaskAttemptContext context)
255 throws IOException, InterruptedException {
256 numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
257 rand = new Random();
258 }
259
260 @Override
261 public boolean nextKeyValue() throws IOException, InterruptedException {
262 return count++ < numNodes;
263 }
264
265 }
266
267 @Override
268 public RecordReader<BytesWritable,NullWritable> createRecordReader(
269 InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
270 GeneratorRecordReader rr = new GeneratorRecordReader();
271 rr.initialize(split, context);
272 return rr;
273 }
274
275 @Override
276 public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
277 int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
278
279 ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
280
281 for (int i = 0; i < numMappers; i++) {
282 splits.add(new GeneratorInputSplit());
283 }
284
285 return splits;
286 }
287 }
288
289
290 static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
291 @Override
292 protected boolean isSplitable(JobContext context, Path filename) {
293 return false;
294 }
295 }
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321 static class GeneratorMapper
322 extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
323 Random rand = new Random();
324
325 byte[][] first = null;
326 byte[][] prev = null;
327 byte[][] current = null;
328 byte[] id;
329 long count = 0;
330 int i;
331 HTable table;
332 long numNodes;
333 long wrap;
334 int width;
335
336 protected void setup(Context context) throws IOException, InterruptedException {
337 id = Bytes.toBytes(UUID.randomUUID().toString());
338 Configuration conf = context.getConfiguration();
339 table = new HTable(conf, getTableName(conf));
340 table.setAutoFlush(false);
341 table.setWriteBufferSize(4 * 1024 * 1024);
342 this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
343 current = new byte[this.width][];
344 int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
345 this.wrap = (long)wrapMultiplier * width;
346 this.numNodes = context.getConfiguration().getLong(
347 GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
348 if (this.numNodes < this.wrap) {
349 this.wrap = this.numNodes;
350 }
351 };
352
353 protected void cleanup(Context context) throws IOException ,InterruptedException {
354 table.close();
355 };
356
357 @Override
358 protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
359 current[i] = new byte[key.getLength()];
360 System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
361 if (++i == current.length) {
362 persist(output, count, prev, current, id);
363 i = 0;
364
365 if (first == null)
366 first = current;
367 prev = current;
368 current = new byte[this.width][];
369
370 count += current.length;
371 output.setStatus("Count " + count);
372
373 if (count % wrap == 0) {
374
375
376 circularLeftShift(first);
377
378 persist(output, -1, prev, first, null);
379
380 first = null;
381 prev = null;
382 }
383 }
384 }
385
386 private static <T> void circularLeftShift(T[] first) {
387 T ez = first[0];
388 for (int i = 0; i < first.length - 1; i++)
389 first[i] = first[i + 1];
390 first[first.length - 1] = ez;
391 }
392
393 private void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
394 throws IOException {
395 for (int i = 0; i < current.length; i++) {
396 Put put = new Put(current[i]);
397 put.add(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
398
399 if (count >= 0) {
400 put.add(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
401 }
402 if (id != null) {
403 put.add(FAMILY_NAME, COLUMN_CLIENT, id);
404 }
405 table.put(put);
406
407 if (i % 1000 == 0) {
408
409 output.progress();
410 }
411 }
412
413 table.flushCommits();
414 }
415 }
416
417 @Override
418 public int run(String[] args) throws Exception {
419 if (args.length < 3) {
420 System.out.println("Usage : " + Generator.class.getSimpleName() +
421 " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>]");
422 System.out.println(" where <num nodes per map> should be a multiple of " +
423 " width*wrap multiplier, 25M by default");
424 return 0;
425 }
426
427 int numMappers = Integer.parseInt(args[0]);
428 long numNodes = Long.parseLong(args[1]);
429 Path tmpOutput = new Path(args[2]);
430 Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
431 Integer wrapMuplitplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
432 return run(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
433 }
434
435 protected void createSchema() throws IOException {
436 HBaseAdmin admin = new HBaseAdmin(getConf());
437 byte[] tableName = getTableName(getConf());
438 if (!admin.tableExists(tableName)) {
439 HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
440 htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
441 admin.createTable(htd);
442 }
443 admin.close();
444 }
445
446 public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
447 Integer width, Integer wrapMuplitplier) throws Exception {
448 LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
449 + ", numNodes=" + numNodes);
450 Job job = new Job(getConf());
451
452 job.setJobName("Random Input Generator");
453 job.setNumReduceTasks(0);
454 job.setJarByClass(getClass());
455
456 job.setInputFormatClass(GeneratorInputFormat.class);
457 job.setOutputKeyClass(BytesWritable.class);
458 job.setOutputValueClass(NullWritable.class);
459
460 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
461
462 job.setMapperClass(Mapper.class);
463
464 FileOutputFormat.setOutputPath(job, tmpOutput);
465 job.setOutputFormatClass(SequenceFileOutputFormat.class);
466
467 boolean success = job.waitForCompletion(true);
468
469 return success ? 0 : 1;
470 }
471
472 public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
473 Integer width, Integer wrapMuplitplier) throws Exception {
474 LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
475 createSchema();
476
477 Job job = new Job(getConf());
478
479 job.setJobName("Link Generator");
480 job.setNumReduceTasks(0);
481 job.setJarByClass(getClass());
482
483 FileInputFormat.setInputPaths(job, tmpOutput);
484 job.setInputFormatClass(OneFilePerMapperSFIF.class);
485 job.setOutputKeyClass(NullWritable.class);
486 job.setOutputValueClass(NullWritable.class);
487
488 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
489
490 job.setMapperClass(GeneratorMapper.class);
491
492 job.setOutputFormatClass(NullOutputFormat.class);
493
494 job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
495 TableMapReduceUtil.addDependencyJars(job);
496 TableMapReduceUtil.initCredentials(job);
497
498 boolean success = job.waitForCompletion(true);
499
500 return success ? 0 : 1;
501 }
502
503 public int run(int numMappers, long numNodes, Path tmpOutput,
504 Integer width, Integer wrapMuplitplier) throws Exception {
505 int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
506 if (ret > 0) {
507 return ret;
508 }
509 return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
510 }
511 }
512
513
514
515
516
517 static class Verify extends Configured implements Tool {
518
519 private static final Log LOG = LogFactory.getLog(Verify.class);
520 private static final BytesWritable DEF = new BytesWritable(NO_KEY);
521
522 private Job job;
523
524 public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
525 private BytesWritable row = new BytesWritable();
526 private BytesWritable ref = new BytesWritable();
527
528 @Override
529 protected void map(ImmutableBytesWritable key, Result value, Context context)
530 throws IOException ,InterruptedException {
531 byte[] rowKey = key.get();
532 row.set(rowKey, 0, rowKey.length);
533 context.write(row, DEF);
534 byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
535 if (prev != null && prev.length > 0) {
536 ref.set(prev, 0, prev.length);
537 context.write(ref, row);
538 } else {
539 LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
540 }
541 }
542 }
543
544 public static enum Counts {
545 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES
546 }
547
548 public static class VerifyReducer extends Reducer<BytesWritable,BytesWritable,Text,Text> {
549 private ArrayList<byte[]> refs = new ArrayList<byte[]>();
550
551 public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
552 throws IOException, InterruptedException {
553
554 int defCount = 0;
555
556 refs.clear();
557 for (BytesWritable type : values) {
558 if (type.getLength() == DEF.getLength()) {
559 defCount++;
560 } else {
561 byte[] bytes = new byte[type.getLength()];
562 System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
563 refs.add(bytes);
564 }
565 }
566
567
568
569 StringBuilder refsSb = null;
570 String keyString = null;
571 if (defCount == 0 || refs.size() != 1) {
572 refsSb = new StringBuilder();
573 String comma = "";
574 for (byte[] ref : refs) {
575 refsSb.append(comma);
576 comma = ",";
577 refsSb.append(Bytes.toStringBinary(ref));
578 }
579 byte[] bytes = new byte[key.getLength()];
580 keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
581 }
582
583 if (defCount == 0 && refs.size() > 0) {
584
585
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
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
598 context.getCounter(Counts.REFERENCED).increment(1);
599 }
600
601 }
602 }
603
604 @Override
605 public int run(String[] args) throws Exception {
606
607 if (args.length != 2) {
608 System.out.println("Usage : " + Verify.class.getSimpleName() + " <output dir> <num reducers>");
609 return 0;
610 }
611
612 String outputDir = args[0];
613 int numReducers = Integer.parseInt(args[1]);
614
615 return run(outputDir, numReducers);
616 }
617
618 public int run(String outputDir, int numReducers) throws Exception {
619 return run(new Path(outputDir), numReducers);
620 }
621
622 public int run(Path outputDir, int numReducers) throws Exception {
623 LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
624
625 job = new Job(getConf());
626
627 job.setJobName("Link Verifier");
628 job.setNumReduceTasks(numReducers);
629 job.setJarByClass(getClass());
630
631 setJobScannerConf(job);
632
633 Scan scan = new Scan();
634 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
635 scan.setCaching(10000);
636 scan.setCacheBlocks(false);
637
638 TableMapReduceUtil.initTableMapperJob(getTableName(getConf()), scan,
639 VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
640
641 job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
642
643 job.setReducerClass(VerifyReducer.class);
644 job.setOutputFormatClass(TextOutputFormat.class);
645 TextOutputFormat.setOutputPath(job, outputDir);
646
647 boolean success = job.waitForCompletion(true);
648
649 return success ? 0 : 1;
650 }
651
652 public boolean verify(long expectedReferenced) throws Exception {
653 if (job == null) {
654 throw new IllegalStateException("You should call run() first");
655 }
656
657 Counters counters = job.getCounters();
658
659 Counter referenced = counters.findCounter(Counts.REFERENCED);
660 Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
661 Counter undefined = counters.findCounter(Counts.UNDEFINED);
662 Counter multiref = counters.findCounter(Counts.EXTRAREFERENCES);
663
664 boolean success = true;
665
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
690
691
692 private static class Loop extends Configured implements Tool {
693
694 private static final Log LOG = LogFactory.getLog(Loop.class);
695
696 protected void runGenerator(int numMappers, long numNodes,
697 String outputDir, Integer width, Integer wrapMuplitplier) throws Exception {
698 Path outputPath = new Path(outputDir);
699 UUID uuid = UUID.randomUUID();
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();
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;
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
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
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
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
886
887
888 while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) {
889 if (!isSpecificStart) {
890 startKey = new byte[ROWKEY_LENGTH];
891 rand.nextBytes(startKey);
892 }
893 CINode node = findStartNode(table, startKey);
894 if (node == null && isSpecificStart) {
895 System.err.printf("Start node not found: %s \n", Bytes.toStringBinary(startKey));
896 }
897 numQueries++;
898 while (node != null && node.prev.length != NO_KEY.length && numQueries < maxQueries) {
899 byte[] prev = node.prev;
900 long t1 = System.currentTimeMillis();
901 node = getNode(prev, table, node);
902 long t2 = System.currentTimeMillis();
903 if (numQueries % logEvery == 0) {
904 System.out.printf("CQ %d: %d %s \n", numQueries, t2 - t1, Bytes.toStringBinary(prev));
905 }
906 numQueries++;
907 if (node == null) {
908 System.err.printf("UNDEFINED NODE %s \n", Bytes.toStringBinary(prev));
909 } else if (node.prev.length == NO_KEY.length) {
910 System.err.printf("TERMINATING NODE %s \n", Bytes.toStringBinary(node.key));
911 }
912 }
913 }
914
915 table.close();
916 return 0;
917 }
918
919 private static CINode findStartNode(HTable table, byte[] startKey) throws IOException {
920 Scan scan = new Scan();
921 scan.setStartRow(startKey);
922 scan.setBatch(1);
923 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
924
925 long t1 = System.currentTimeMillis();
926 ResultScanner scanner = table.getScanner(scan);
927 Result result = scanner.next();
928 long t2 = System.currentTimeMillis();
929 scanner.close();
930
931 if ( result != null) {
932 CINode node = getCINode(result, new CINode());
933 System.out.printf("FSR %d %s\n", t2 - t1, Bytes.toStringBinary(node.key));
934 return node;
935 }
936
937 System.out.println("FSR " + (t2 - t1));
938
939 return null;
940 }
941
942 private CINode getNode(byte[] row, HTable table, CINode node) throws IOException {
943 Get get = new Get(row);
944 get.addColumn(FAMILY_NAME, COLUMN_PREV);
945 Result result = table.get(get);
946 return getCINode(result, node);
947 }
948 }
949
950 private static byte[] getTableName(Configuration conf) {
951 return Bytes.toBytes(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
952 }
953
954 private static CINode getCINode(Result result, CINode node) {
955 node.key = new byte[result.getRow().length];
956 System.arraycopy(result.getRow(), 0, node.key, 0, node.key.length);
957 if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
958 byte[] value = result.getValue(FAMILY_NAME, COLUMN_PREV);
959 node.prev = new byte[value.length];
960 System.arraycopy(value, 0, node.prev, 0, node.prev.length);
961 } else {
962 node.prev = NO_KEY;
963 }
964 if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
965 node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
966 } else {
967 node.count = -1;
968 }
969 if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
970 node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
971 } else {
972 node.client = "";
973 }
974 return node;
975 }
976
977 private IntegrationTestingUtility util;
978
979 @Before
980 public void setUp() throws Exception {
981 util = getTestingUtil();
982 util.initializeCluster(3);
983 this.setConf(util.getConfiguration());
984 }
985
986 @After
987 public void tearDown() throws Exception {
988 util.restoreCluster();
989 }
990
991 @Test
992 public void testContinuousIngest() throws IOException, Exception {
993
994 int ret = ToolRunner.run(getTestingUtil().getConfiguration(), new Loop(),
995 new String[] {"1", "1", "2000000",
996 getTestDir("IntegrationTestBigLinkedList", "testContinuousIngest").toString(), "1"});
997 org.junit.Assert.assertEquals(0, ret);
998 }
999
1000 public Path getTestDir(String testName, String subdir) throws IOException {
1001
1002 FileSystem fs = FileSystem.get(getConf());
1003 Path base = new Path(fs.getWorkingDirectory(), "test-data");
1004 String randomStr = UUID.randomUUID().toString();
1005 Path testDir = new Path(base, randomStr);
1006 fs.deleteOnExit(testDir);
1007
1008 return new Path(new Path(testDir, testName), subdir);
1009 }
1010
1011 private IntegrationTestingUtility getTestingUtil() {
1012 if (this.util == null) {
1013 if (getConf() == null) {
1014 this.util = new IntegrationTestingUtility();
1015 } else {
1016 this.util = new IntegrationTestingUtility(getConf());
1017 }
1018 }
1019 return util;
1020 }
1021
1022 private int printUsage() {
1023 System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1024 System.err.println(" where COMMAND is one of:");
1025 System.err.println("");
1026 System.err.println(" Generator A map only job that generates data.");
1027 System.err.println(" Verify A map reduce job that looks for holes");
1028 System.err.println(" Look at the counts after running");
1029 System.err.println(" REFERENCED and UNREFERENCED are ok");
1030 System.err.println(" any UNDEFINED counts are bad. Do not");
1031 System.err.println(" run at the same time as the Generator.");
1032 System.err.println(" Walker A standalong program that starts ");
1033 System.err.println(" following a linked list and emits");
1034 System.err.println(" timing info.");
1035 System.err.println(" Print A standalone program that prints nodes");
1036 System.err.println(" in the linked list.");
1037 System.err.println(" Delete A standalone program that deletes a·");
1038 System.err.println(" single node.");
1039 System.err.println(" Loop A program to Loop through Generator and");
1040 System.err.println(" Verify steps");
1041 System.err.println("\t ");
1042 return 1;
1043 }
1044
1045 @Override
1046 public int run(String[] args) throws Exception {
1047
1048 if (args.length < 1) {
1049 return printUsage();
1050 }
1051 Tool tool = null;
1052 if (args[0].equals("Generator")) {
1053 tool = new Generator();
1054 } else if (args[0].equals("Verify")) {
1055 tool = new Verify();
1056 } else if (args[0].equals("Loop")) {
1057 tool = new Loop();
1058 } else if (args[0].equals("Walker")) {
1059 tool = new Walker();
1060 } else if (args[0].equals("Print")) {
1061 tool = new Print();
1062 } else if (args[0].equals("Delete")) {
1063 tool = new Delete();
1064 } else {
1065 return printUsage();
1066 }
1067
1068 args = Arrays.copyOfRange(args, 1, args.length);
1069 return ToolRunner.run(getConf(), tool, args);
1070 }
1071
1072 public static void main(String[] args) throws Exception {
1073 int ret = ToolRunner.run(HBaseConfiguration.create(), new IntegrationTestBigLinkedList(), args);
1074 System.exit(ret);
1075 }
1076
1077 private static void setJobConf(Job job, int numMappers, long numNodes,
1078 Integer width, Integer wrapMuplitplier) {
1079 job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1080 job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1081 if (width != null) {
1082 job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width.intValue());
1083 }
1084 if (wrapMuplitplier != null) {
1085 job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMuplitplier.intValue());
1086 }
1087 }
1088
1089 private static void setJobScannerConf(Job job) {
1090
1091 job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1092 job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1093 }
1094 }