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.io.ImmutableBytesWritable;
56 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
57 import org.apache.hadoop.hbase.mapreduce.TableMapper;
58 import org.apache.hadoop.hbase.util.Bytes;
59 import org.apache.hadoop.io.BytesWritable;
60 import org.apache.hadoop.io.NullWritable;
61 import org.apache.hadoop.io.Text;
62 import org.apache.hadoop.io.Writable;
63 import org.apache.hadoop.mapreduce.Counter;
64 import org.apache.hadoop.mapreduce.Counters;
65 import org.apache.hadoop.mapreduce.InputFormat;
66 import org.apache.hadoop.mapreduce.InputSplit;
67 import org.apache.hadoop.mapreduce.Job;
68 import org.apache.hadoop.mapreduce.JobContext;
69 import org.apache.hadoop.mapreduce.Mapper;
70 import org.apache.hadoop.mapreduce.RecordReader;
71 import org.apache.hadoop.mapreduce.Reducer;
72 import org.apache.hadoop.mapreduce.TaskAttemptContext;
73 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
74 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
75 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
76 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
77 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
78 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
79 import org.apache.hadoop.util.Tool;
80 import org.apache.hadoop.util.ToolRunner;
81 import org.junit.After;
82 import org.junit.Before;
83 import org.junit.Test;
84 import org.junit.experimental.categories.Category;
85
86
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 @Category(IntegrationTests.class)
150 public class IntegrationTestBigLinkedList extends Configured implements Tool {
151 private static final byte[] NO_KEY = new byte[1];
152
153 private static final String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
154
155 private static final String DEFAULT_TABLE_NAME = "IntegrationTestBigLinkedList";
156
157 private static byte[] FAMILY_NAME = Bytes.toBytes("meta");
158
159
160 private static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
161
162
163 private static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
164
165
166 private static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
167
168
169 private static final String GENERATOR_NUM_ROWS_PER_MAP_KEY
170 = "IntegrationTestBigLinkedList.generator.num_rows";
171
172 private static final String GENERATOR_NUM_MAPPERS_KEY
173 = "IntegrationTestBigLinkedList.generator.map.tasks";
174
175 private static final String GENERATOR_WIDTH_KEY
176 = "IntegrationTestBigLinkedList.generator.width";
177
178 private static final String GENERATOR_WRAP_KEY
179 = "IntegrationTestBigLinkedList.generator.wrap";
180
181 protected int NUM_SLAVES_BASE = 3;
182
183 private static final int WIDTH_DEFAULT = 1000000;
184 private static final int WRAP_DEFAULT = 25;
185
186 private static final int ROWKEY_LENGTH = 16;
187
188 static class CINode {
189 byte[] key;
190 byte[] prev;
191
192 String client;
193 long count;
194 }
195
196
197
198
199 static class Generator extends Configured implements Tool {
200
201 private static final Log LOG = LogFactory.getLog(Generator.class);
202
203 public static enum Counts {
204 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT
205 }
206
207 static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
208 static class GeneratorInputSplit extends InputSplit implements Writable {
209 @Override
210 public long getLength() throws IOException, InterruptedException {
211 return 1;
212 }
213 @Override
214 public String[] getLocations() throws IOException, InterruptedException {
215 return new String[0];
216 }
217 @Override
218 public void readFields(DataInput arg0) throws IOException {
219 }
220 @Override
221 public void write(DataOutput arg0) throws IOException {
222 }
223 }
224
225 static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
226 private long count;
227 private long numNodes;
228 private Random rand;
229
230 @Override
231 public void close() throws IOException {
232 }
233
234 @Override
235 public BytesWritable getCurrentKey() throws IOException, InterruptedException {
236 byte[] bytes = new byte[ROWKEY_LENGTH];
237 rand.nextBytes(bytes);
238 return new BytesWritable(bytes);
239 }
240
241 @Override
242 public NullWritable getCurrentValue() throws IOException, InterruptedException {
243 return NullWritable.get();
244 }
245
246 @Override
247 public float getProgress() throws IOException, InterruptedException {
248 return (float)(count / (double)numNodes);
249 }
250
251 @Override
252 public void initialize(InputSplit arg0, TaskAttemptContext context)
253 throws IOException, InterruptedException {
254 numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
255 rand = new Random();
256 }
257
258 @Override
259 public boolean nextKeyValue() throws IOException, InterruptedException {
260 return count++ < numNodes;
261 }
262
263 }
264
265 @Override
266 public RecordReader<BytesWritable,NullWritable> createRecordReader(
267 InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
268 GeneratorRecordReader rr = new GeneratorRecordReader();
269 rr.initialize(split, context);
270 return rr;
271 }
272
273 @Override
274 public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
275 int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
276
277 ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
278
279 for (int i = 0; i < numMappers; i++) {
280 splits.add(new GeneratorInputSplit());
281 }
282
283 return splits;
284 }
285 }
286
287
288 static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
289 @Override
290 protected boolean isSplitable(JobContext context, Path filename) {
291 return false;
292 }
293 }
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 static class GeneratorMapper
320 extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
321 Random rand = new Random();
322
323 byte[][] first = null;
324 byte[][] prev = null;
325 byte[][] current = null;
326 byte[] id;
327 long count = 0;
328 int i;
329 HTable table;
330 long numNodes;
331 long wrap;
332 int width;
333
334 protected void setup(Context context) throws IOException, InterruptedException {
335 id = Bytes.toBytes(UUID.randomUUID().toString());
336 Configuration conf = context.getConfiguration();
337 table = new HTable(conf, getTableName(conf));
338 table.setAutoFlush(false);
339 table.setWriteBufferSize(4 * 1024 * 1024);
340 this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
341 current = new byte[this.width][];
342 int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
343 this.wrap = (long)wrapMultiplier * width;
344 this.numNodes = context.getConfiguration().getLong(
345 GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
346 if (this.numNodes < this.wrap) {
347 this.wrap = this.numNodes;
348 }
349 };
350
351 protected void cleanup(Context context) throws IOException ,InterruptedException {
352 table.close();
353 };
354
355 @Override
356 protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
357 current[i] = new byte[key.getLength()];
358 System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
359 if (++i == current.length) {
360 persist(output, count, prev, current, id);
361 i = 0;
362
363 if (first == null)
364 first = current;
365 prev = current;
366 current = new byte[this.width][];
367
368 count += current.length;
369 output.setStatus("Count " + count);
370
371 if (count % wrap == 0) {
372
373
374 circularLeftShift(first);
375
376 persist(output, -1, prev, first, null);
377
378 first = null;
379 prev = null;
380 }
381 }
382 }
383
384 private static <T> void circularLeftShift(T[] first) {
385 T ez = first[0];
386 for (int i = 0; i < first.length - 1; i++)
387 first[i] = first[i + 1];
388 first[first.length - 1] = ez;
389 }
390
391 private void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
392 throws IOException {
393 for (int i = 0; i < current.length; i++) {
394 Put put = new Put(current[i]);
395 put.add(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
396
397 if (count >= 0) {
398 put.add(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
399 }
400 if (id != null) {
401 put.add(FAMILY_NAME, COLUMN_CLIENT, id);
402 }
403 table.put(put);
404
405 if (i % 1000 == 0) {
406
407 output.progress();
408 }
409 }
410
411 table.flushCommits();
412 }
413 }
414
415 @Override
416 public int run(String[] args) throws Exception {
417 if (args.length < 3) {
418 System.out.println("Usage : " + Generator.class.getSimpleName() +
419 " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>]");
420 System.out.println(" where <num nodes per map> should be a multiple of " +
421 " width*wrap multiplier, 25M by default");
422 return 0;
423 }
424
425 int numMappers = Integer.parseInt(args[0]);
426 long numNodes = Long.parseLong(args[1]);
427 Path tmpOutput = new Path(args[2]);
428 Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
429 Integer wrapMuplitplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
430 return run(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
431 }
432
433 protected void createSchema() throws IOException {
434 HBaseAdmin admin = new HBaseAdmin(getConf());
435 byte[] tableName = getTableName(getConf());
436 if (!admin.tableExists(tableName)) {
437 HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
438 htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
439 admin.createTable(htd);
440 }
441 admin.close();
442 }
443
444 public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
445 Integer width, Integer wrapMuplitplier) throws Exception {
446 LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
447 + ", numNodes=" + numNodes);
448 Job job = new Job(getConf());
449
450 job.setJobName("Random Input Generator");
451 job.setNumReduceTasks(0);
452 job.setJarByClass(getClass());
453
454 job.setInputFormatClass(GeneratorInputFormat.class);
455 job.setOutputKeyClass(BytesWritable.class);
456 job.setOutputValueClass(NullWritable.class);
457
458 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
459
460 job.setMapperClass(Mapper.class);
461
462 FileOutputFormat.setOutputPath(job, tmpOutput);
463 job.setOutputFormatClass(SequenceFileOutputFormat.class);
464
465 boolean success = job.waitForCompletion(true);
466
467 return success ? 0 : 1;
468 }
469
470 public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
471 Integer width, Integer wrapMuplitplier) throws Exception {
472 LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
473 createSchema();
474
475 Job job = new Job(getConf());
476
477 job.setJobName("Link Generator");
478 job.setNumReduceTasks(0);
479 job.setJarByClass(getClass());
480
481 FileInputFormat.setInputPaths(job, tmpOutput);
482 job.setInputFormatClass(OneFilePerMapperSFIF.class);
483 job.setOutputKeyClass(NullWritable.class);
484 job.setOutputValueClass(NullWritable.class);
485
486 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
487
488 job.setMapperClass(GeneratorMapper.class);
489
490 job.setOutputFormatClass(NullOutputFormat.class);
491
492 job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
493 TableMapReduceUtil.addDependencyJars(job);
494 TableMapReduceUtil.initCredentials(job);
495
496 boolean success = job.waitForCompletion(true);
497
498 return success ? 0 : 1;
499 }
500
501 public int run(int numMappers, long numNodes, Path tmpOutput,
502 Integer width, Integer wrapMuplitplier) throws Exception {
503 int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
504 if (ret > 0) {
505 return ret;
506 }
507 return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
508 }
509 }
510
511
512
513
514
515 static class Verify extends Configured implements Tool {
516
517 private static final Log LOG = LogFactory.getLog(Verify.class);
518 private static final BytesWritable DEF = new BytesWritable(NO_KEY);
519
520 private Job job;
521
522 public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
523 private BytesWritable row = new BytesWritable();
524 private BytesWritable ref = new BytesWritable();
525
526 @Override
527 protected void map(ImmutableBytesWritable key, Result value, Context context)
528 throws IOException ,InterruptedException {
529 byte[] rowKey = key.get();
530 row.set(rowKey, 0, rowKey.length);
531 context.write(row, DEF);
532 byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
533 if (prev != null && prev.length > 0) {
534 ref.set(prev, 0, prev.length);
535 context.write(ref, row);
536 } else {
537 LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
538 }
539 }
540 }
541
542 public static enum Counts {
543 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES
544 }
545
546 public static class VerifyReducer extends Reducer<BytesWritable,BytesWritable,Text,Text> {
547 private ArrayList<byte[]> refs = new ArrayList<byte[]>();
548
549 public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
550 throws IOException, InterruptedException {
551
552 int defCount = 0;
553
554 refs.clear();
555 for (BytesWritable type : values) {
556 if (type.getLength() == DEF.getLength()) {
557 defCount++;
558 } else {
559 byte[] bytes = new byte[type.getLength()];
560 System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
561 refs.add(bytes);
562 }
563 }
564
565
566
567 StringBuilder refsSb = null;
568 String keyString = null;
569 if (defCount == 0 || refs.size() != 1) {
570 refsSb = new StringBuilder();
571 String comma = "";
572 for (byte[] ref : refs) {
573 refsSb.append(comma);
574 comma = ",";
575 refsSb.append(Bytes.toStringBinary(ref));
576 }
577 byte[] bytes = new byte[key.getLength()];
578 keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
579 }
580
581 if (defCount == 0 && refs.size() > 0) {
582
583
584 context.write(new Text(keyString), new Text(refsSb.toString()));
585 context.getCounter(Counts.UNDEFINED).increment(1);
586 } else if (defCount > 0 && refs.size() == 0) {
587
588 context.write(new Text(keyString), new Text("none"));
589 context.getCounter(Counts.UNREFERENCED).increment(1);
590 } else {
591 if (refs.size() > 1) {
592 context.write(new Text(keyString), new Text(refsSb.toString()));
593 context.getCounter(Counts.EXTRAREFERENCES).increment(refs.size() - 1);
594 }
595
596 context.getCounter(Counts.REFERENCED).increment(1);
597 }
598
599 }
600 }
601
602 @Override
603 public int run(String[] args) throws Exception {
604
605 if (args.length != 2) {
606 System.out.println("Usage : " + Verify.class.getSimpleName() + " <output dir> <num reducers>");
607 return 0;
608 }
609
610 String outputDir = args[0];
611 int numReducers = Integer.parseInt(args[1]);
612
613 return run(outputDir, numReducers);
614 }
615
616 public int run(String outputDir, int numReducers) throws Exception {
617 return run(new Path(outputDir), numReducers);
618 }
619
620 public int run(Path outputDir, int numReducers) throws Exception {
621 LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
622
623 job = new Job(getConf());
624
625 job.setJobName("Link Verifier");
626 job.setNumReduceTasks(numReducers);
627 job.setJarByClass(getClass());
628
629 Scan scan = new Scan();
630 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
631 scan.setCaching(10000);
632 scan.setCacheBlocks(false);
633
634 TableMapReduceUtil.initTableMapperJob(getTableName(getConf()), scan,
635 VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
636
637 job.getConfiguration().setBoolean("mapred.map.tasks.speculative.execution", false);
638
639 job.setReducerClass(VerifyReducer.class);
640 job.setOutputFormatClass(TextOutputFormat.class);
641 TextOutputFormat.setOutputPath(job, outputDir);
642
643 boolean success = job.waitForCompletion(true);
644
645 return success ? 0 : 1;
646 }
647
648 public boolean verify(long expectedReferenced) throws Exception {
649 if (job == null) {
650 throw new IllegalStateException("You should call run() first");
651 }
652
653 Counters counters = job.getCounters();
654
655 Counter referenced = counters.findCounter(Counts.REFERENCED);
656 Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
657 Counter undefined = counters.findCounter(Counts.UNDEFINED);
658 Counter multiref = counters.findCounter(Counts.EXTRAREFERENCES);
659
660 boolean success = true;
661
662 if (expectedReferenced != referenced.getValue()) {
663 LOG.error("Expected referenced count does not match with actual referenced count. " +
664 "expected referenced=" + expectedReferenced + " ,actual=" + referenced.getValue());
665 success = false;
666 }
667
668 if (unreferenced.getValue() > 0) {
669 boolean couldBeMultiRef = (multiref.getValue() == unreferenced.getValue());
670 LOG.error("Unreferenced nodes were not expected. Unreferenced count=" + unreferenced.getValue()
671 + (couldBeMultiRef ? "; could be due to duplicate random numbers" : ""));
672 success = false;
673 }
674
675 if (undefined.getValue() > 0) {
676 LOG.error("Found an undefined node. Undefined count=" + undefined.getValue());
677 success = false;
678 }
679
680 return success;
681 }
682 }
683
684
685
686
687
688 private static class Loop extends Configured implements Tool {
689
690 private static final Log LOG = LogFactory.getLog(Loop.class);
691
692 protected void runGenerator(int numMappers, long numNodes,
693 String outputDir, Integer width, Integer wrapMuplitplier) throws Exception {
694 Path outputPath = new Path(outputDir);
695 UUID uuid = UUID.randomUUID();
696 Path generatorOutput = new Path(outputPath, uuid.toString());
697
698 Generator generator = new Generator();
699 generator.setConf(getConf());
700 int retCode = generator.run(numMappers, numNodes, generatorOutput, width, wrapMuplitplier);
701 if (retCode > 0) {
702 throw new RuntimeException("Generator failed with return code: " + retCode);
703 }
704 }
705
706 protected void runVerify(String outputDir, int numReducers, long expectedNumNodes) throws Exception {
707 Path outputPath = new Path(outputDir);
708 UUID uuid = UUID.randomUUID();
709 Path iterationOutput = new Path(outputPath, uuid.toString());
710
711 Verify verify = new Verify();
712 verify.setConf(getConf());
713 int retCode = verify.run(iterationOutput, numReducers);
714 if (retCode > 0) {
715 throw new RuntimeException("Verify.run failed with return code: " + retCode);
716 }
717
718 boolean verifySuccess = verify.verify(expectedNumNodes);
719 if (!verifySuccess) {
720 throw new RuntimeException("Verify.verify failed");
721 }
722
723 LOG.info("Verify finished with succees. Total nodes=" + expectedNumNodes);
724 }
725
726 @Override
727 public int run(String[] args) throws Exception {
728 if (args.length < 5) {
729 System.err.println("Usage: Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers> [<width> <wrap multiplier>]");
730 return 1;
731 }
732 LOG.info("Running Loop with args:" + Arrays.deepToString(args));
733
734 int numIterations = Integer.parseInt(args[0]);
735 int numMappers = Integer.parseInt(args[1]);
736 long numNodes = Long.parseLong(args[2]);
737 String outputDir = args[3];
738 int numReducers = Integer.parseInt(args[4]);
739 Integer width = (args.length < 6) ? null : Integer.parseInt(args[5]);
740 Integer wrapMuplitplier = (args.length < 7) ? null : Integer.parseInt(args[6]);
741
742 long expectedNumNodes = 0;
743
744 if (numIterations < 0) {
745 numIterations = Integer.MAX_VALUE;
746 }
747
748 for (int i = 0; i < numIterations; i++) {
749 LOG.info("Starting iteration = " + i);
750 runGenerator(numMappers, numNodes, outputDir, width, wrapMuplitplier);
751 expectedNumNodes += numMappers * numNodes;
752
753 runVerify(outputDir, numReducers, expectedNumNodes);
754 }
755
756 return 0;
757 }
758 }
759
760
761
762
763 private static class Print extends Configured implements Tool {
764 public int run(String[] args) throws Exception {
765 Options options = new Options();
766 options.addOption("s", "start", true, "start key");
767 options.addOption("e", "end", true, "end key");
768 options.addOption("l", "limit", true, "number to print");
769
770 GnuParser parser = new GnuParser();
771 CommandLine cmd = null;
772 try {
773 cmd = parser.parse(options, args);
774 if (cmd.getArgs().length != 0) {
775 throw new ParseException("Command takes no arguments");
776 }
777 } catch (ParseException e) {
778 System.err.println("Failed to parse command line " + e.getMessage());
779 System.err.println();
780 HelpFormatter formatter = new HelpFormatter();
781 formatter.printHelp(getClass().getSimpleName(), options);
782 System.exit(-1);
783 }
784
785 HTable table = new HTable(getConf(), getTableName(getConf()));
786
787 Scan scan = new Scan();
788 scan.setBatch(10000);
789
790 if (cmd.hasOption("s"))
791 scan.setStartRow(Bytes.toBytesBinary(cmd.getOptionValue("s")));
792
793 if (cmd.hasOption("e"))
794 scan.setStopRow(Bytes.toBytesBinary(cmd.getOptionValue("e")));
795
796 int limit = 0;
797 if (cmd.hasOption("l"))
798 limit = Integer.parseInt(cmd.getOptionValue("l"));
799 else
800 limit = 100;
801
802 ResultScanner scanner = table.getScanner(scan);
803
804 CINode node = new CINode();
805 Result result = scanner.next();
806 int count = 0;
807 while (result != null && count++ < limit) {
808 node = getCINode(result, node);
809 System.out.printf("%s:%s:%012d:%s\n", Bytes.toStringBinary(node.key),
810 Bytes.toStringBinary(node.prev), node.count, node.client);
811 result = scanner.next();
812 }
813 scanner.close();
814 table.close();
815
816 return 0;
817 }
818 }
819
820
821
822
823 private static class Delete extends Configured implements Tool {
824 public int run(String[] args) throws Exception {
825 if (args.length != 1) {
826 System.out.println("Usage : " + Delete.class.getSimpleName() + " <node to delete>");
827 return 0;
828 }
829 byte[] val = Bytes.toBytesBinary(args[0]);
830
831 org.apache.hadoop.hbase.client.Delete delete
832 = new org.apache.hadoop.hbase.client.Delete(val);
833
834 HTable table = new HTable(getConf(), getTableName(getConf()));
835
836 table.delete(delete);
837 table.flushCommits();
838 table.close();
839
840 System.out.println("Delete successful");
841 return 0;
842 }
843 }
844
845
846
847
848 private static class Walker extends Configured implements Tool {
849 public int run(String[] args) throws IOException {
850 Options options = new Options();
851 options.addOption("n", "num", true, "number of queries");
852 options.addOption("s", "start", true, "key to start at, binary string");
853 options.addOption("l", "logevery", true, "log every N queries");
854
855 GnuParser parser = new GnuParser();
856 CommandLine cmd = null;
857 try {
858 cmd = parser.parse(options, args);
859 if (cmd.getArgs().length != 0) {
860 throw new ParseException("Command takes no arguments");
861 }
862 } catch (ParseException e) {
863 System.err.println("Failed to parse command line " + e.getMessage());
864 System.err.println();
865 HelpFormatter formatter = new HelpFormatter();
866 formatter.printHelp(getClass().getSimpleName(), options);
867 System.exit(-1);
868 }
869
870 long maxQueries = Long.MAX_VALUE;
871 if (cmd.hasOption('n')) {
872 maxQueries = Long.parseLong(cmd.getOptionValue("n"));
873 }
874 Random rand = new Random();
875 boolean isSpecificStart = cmd.hasOption('s');
876 byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
877 int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1;
878
879 HTable table = new HTable(getConf(), getTableName(getConf()));
880 long numQueries = 0;
881
882
883
884 while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) {
885 if (!isSpecificStart) {
886 startKey = new byte[ROWKEY_LENGTH];
887 rand.nextBytes(startKey);
888 }
889 CINode node = findStartNode(table, startKey);
890 if (node == null && isSpecificStart) {
891 System.err.printf("Start node not found: %s \n", Bytes.toStringBinary(startKey));
892 }
893 numQueries++;
894 while (node != null && node.prev.length != NO_KEY.length && numQueries < maxQueries) {
895 byte[] prev = node.prev;
896 long t1 = System.currentTimeMillis();
897 node = getNode(prev, table, node);
898 long t2 = System.currentTimeMillis();
899 if (numQueries % logEvery == 0) {
900 System.out.printf("CQ %d: %d %s \n", numQueries, t2 - t1, Bytes.toStringBinary(prev));
901 }
902 numQueries++;
903 if (node == null) {
904 System.err.printf("UNDEFINED NODE %s \n", Bytes.toStringBinary(prev));
905 } else if (node.prev.length == NO_KEY.length) {
906 System.err.printf("TERMINATING NODE %s \n", Bytes.toStringBinary(node.key));
907 }
908 }
909 }
910
911 table.close();
912 return 0;
913 }
914
915 private static CINode findStartNode(HTable table, byte[] startKey) throws IOException {
916 Scan scan = new Scan();
917 scan.setStartRow(startKey);
918 scan.setBatch(1);
919 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
920
921 long t1 = System.currentTimeMillis();
922 ResultScanner scanner = table.getScanner(scan);
923 Result result = scanner.next();
924 long t2 = System.currentTimeMillis();
925 scanner.close();
926
927 if ( result != null) {
928 CINode node = getCINode(result, new CINode());
929 System.out.printf("FSR %d %s\n", t2 - t1, Bytes.toStringBinary(node.key));
930 return node;
931 }
932
933 System.out.println("FSR " + (t2 - t1));
934
935 return null;
936 }
937
938 private CINode getNode(byte[] row, HTable table, CINode node) throws IOException {
939 Get get = new Get(row);
940 get.addColumn(FAMILY_NAME, COLUMN_PREV);
941 Result result = table.get(get);
942 return getCINode(result, node);
943 }
944 }
945
946 private static byte[] getTableName(Configuration conf) {
947 return Bytes.toBytes(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
948 }
949
950 private static CINode getCINode(Result result, CINode node) {
951 node.key = new byte[result.getRow().length];
952 System.arraycopy(result.getRow(), 0, node.key, 0, node.key.length);
953 if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
954 byte[] value = result.getValue(FAMILY_NAME, COLUMN_PREV);
955 node.prev = new byte[value.length];
956 System.arraycopy(value, 0, node.prev, 0, node.prev.length);
957 } else {
958 node.prev = NO_KEY;
959 }
960 if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
961 node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
962 } else {
963 node.count = -1;
964 }
965 if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
966 node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
967 } else {
968 node.client = "";
969 }
970 return node;
971 }
972
973 private IntegrationTestingUtility util;
974
975 @Before
976 public void setUp() throws Exception {
977 util = getTestingUtil();
978 util.initializeCluster(3);
979 this.setConf(util.getConfiguration());
980 }
981
982 @After
983 public void tearDown() throws Exception {
984 util.restoreCluster();
985 }
986
987 @Test
988 public void testContinuousIngest() throws IOException, Exception {
989
990 int ret = ToolRunner.run(getTestingUtil().getConfiguration(), new Loop(),
991 new String[] {"1", "1", "2000000",
992 getTestDir("IntegrationTestBigLinkedList", "testContinuousIngest").toString(), "1"});
993 org.junit.Assert.assertEquals(0, ret);
994 }
995
996 public Path getTestDir(String testName, String subdir) throws IOException {
997
998 FileSystem fs = FileSystem.get(getConf());
999 Path base = new Path(fs.getWorkingDirectory(), "test-data");
1000 String randomStr = UUID.randomUUID().toString();
1001 Path testDir = new Path(base, randomStr);
1002 fs.deleteOnExit(testDir);
1003
1004 return new Path(new Path(testDir, testName), subdir);
1005 }
1006
1007 private IntegrationTestingUtility getTestingUtil() {
1008 if (this.util == null) {
1009 if (getConf() == null) {
1010 this.util = new IntegrationTestingUtility();
1011 } else {
1012 this.util = new IntegrationTestingUtility(getConf());
1013 }
1014 }
1015 return util;
1016 }
1017
1018 private int printUsage() {
1019 System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1020 System.err.println(" where COMMAND is one of:");
1021 System.err.println("");
1022 System.err.println(" Generator A map only job that generates data.");
1023 System.err.println(" Verify A map reduce job that looks for holes");
1024 System.err.println(" Look at the counts after running");
1025 System.err.println(" REFERENCED and UNREFERENCED are ok");
1026 System.err.println(" any UNDEFINED counts are bad. Do not");
1027 System.err.println(" run at the same time as the Generator.");
1028 System.err.println(" Walker A standalong program that starts ");
1029 System.err.println(" following a linked list and emits");
1030 System.err.println(" timing info.");
1031 System.err.println(" Print A standalone program that prints nodes");
1032 System.err.println(" in the linked list.");
1033 System.err.println(" Delete A standalone program that deletes a·");
1034 System.err.println(" single node.");
1035 System.err.println(" Loop A program to Loop through Generator and");
1036 System.err.println(" Verify steps");
1037 System.err.println("\t ");
1038 return 1;
1039 }
1040
1041 @Override
1042 public int run(String[] args) throws Exception {
1043
1044 if (args.length < 1) {
1045 return printUsage();
1046 }
1047 Tool tool = null;
1048 if (args[0].equals("Generator")) {
1049 tool = new Generator();
1050 } else if (args[0].equals("Verify")) {
1051 tool = new Verify();
1052 } else if (args[0].equals("Loop")) {
1053 tool = new Loop();
1054 } else if (args[0].equals("Walker")) {
1055 tool = new Walker();
1056 } else if (args[0].equals("Print")) {
1057 tool = new Print();
1058 } else if (args[0].equals("Delete")) {
1059 tool = new Delete();
1060 } else {
1061 return printUsage();
1062 }
1063
1064 args = Arrays.copyOfRange(args, 1, args.length);
1065 return ToolRunner.run(getConf(), tool, args);
1066 }
1067
1068 public static void main(String[] args) throws Exception {
1069 int ret = ToolRunner.run(HBaseConfiguration.create(), new IntegrationTestBigLinkedList(), args);
1070 System.exit(ret);
1071 }
1072
1073 private static void setJobConf(Job job, int numMappers, long numNodes,
1074 Integer width, Integer wrapMuplitplier) {
1075 job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1076 job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1077 if (width != null) {
1078 job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width.intValue());
1079 }
1080 if (wrapMuplitplier != null) {
1081 job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMuplitplier.intValue());
1082 }
1083 }
1084 }