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