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.FileNotFoundException;
25 import java.io.InterruptedIOException;
26 import java.util.ArrayList;
27 import java.util.Arrays;
28 import java.util.Iterator;
29 import java.util.List;
30 import java.util.Random;
31 import java.util.Set;
32 import java.util.SortedSet;
33 import java.util.TreeSet;
34 import java.util.UUID;
35 import java.util.concurrent.atomic.AtomicInteger;
36
37 import org.apache.commons.cli.CommandLine;
38 import org.apache.commons.cli.GnuParser;
39 import org.apache.commons.cli.HelpFormatter;
40 import org.apache.commons.cli.Options;
41 import org.apache.commons.cli.ParseException;
42 import org.apache.commons.logging.Log;
43 import org.apache.commons.logging.LogFactory;
44 import org.apache.hadoop.conf.Configuration;
45 import org.apache.hadoop.conf.Configured;
46 import org.apache.hadoop.fs.FileSystem;
47 import org.apache.hadoop.fs.LocatedFileStatus;
48 import org.apache.hadoop.fs.Path;
49 import org.apache.hadoop.fs.RemoteIterator;
50 import org.apache.hadoop.hbase.Cell;
51 import org.apache.hadoop.hbase.HBaseConfiguration;
52 import org.apache.hadoop.hbase.HBaseTestingUtility;
53 import org.apache.hadoop.hbase.HColumnDescriptor;
54 import org.apache.hadoop.hbase.HConstants;
55 import org.apache.hadoop.hbase.HRegionLocation;
56 import org.apache.hadoop.hbase.HTableDescriptor;
57 import org.apache.hadoop.hbase.client.HTable;
58 import org.apache.hadoop.hbase.client.Table;
59 import org.apache.hadoop.hbase.IntegrationTestBase;
60 import org.apache.hadoop.hbase.IntegrationTestingUtility;
61 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
62 import org.apache.hadoop.hbase.fs.HFileSystem;
63 import org.apache.hadoop.hbase.MasterNotRunningException;
64 import org.apache.hadoop.hbase.TableName;
65 import org.apache.hadoop.hbase.client.Admin;
66 import org.apache.hadoop.hbase.client.BufferedMutator;
67 import org.apache.hadoop.hbase.client.BufferedMutatorParams;
68 import org.apache.hadoop.hbase.client.Connection;
69 import org.apache.hadoop.hbase.client.ConnectionFactory;
70 import org.apache.hadoop.hbase.client.Get;
71 import org.apache.hadoop.hbase.client.HBaseAdmin;
72 import org.apache.hadoop.hbase.client.Mutation;
73 import org.apache.hadoop.hbase.client.Put;
74 import org.apache.hadoop.hbase.client.RegionLocator;
75 import org.apache.hadoop.hbase.client.Result;
76 import org.apache.hadoop.hbase.client.ResultScanner;
77 import org.apache.hadoop.hbase.client.Scan;
78 import org.apache.hadoop.hbase.client.ScannerCallable;
79 import org.apache.hadoop.hbase.client.Table;
80 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
81 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
82 import org.apache.hadoop.hbase.mapreduce.TableMapper;
83 import org.apache.hadoop.hbase.mapreduce.TableRecordReaderImpl;
84 import org.apache.hadoop.hbase.mapreduce.WALPlayer;
85 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
86 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
87 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
88 import org.apache.hadoop.hbase.util.Bytes;
89 import org.apache.hadoop.hbase.util.RegionSplitter;
90 import org.apache.hadoop.hbase.wal.WALKey;
91 import org.apache.hadoop.io.BytesWritable;
92 import org.apache.hadoop.io.NullWritable;
93 import org.apache.hadoop.io.Writable;
94 import org.apache.hadoop.mapreduce.Counter;
95 import org.apache.hadoop.mapreduce.CounterGroup;
96 import org.apache.hadoop.mapreduce.Counters;
97 import org.apache.hadoop.mapreduce.InputFormat;
98 import org.apache.hadoop.mapreduce.InputSplit;
99 import org.apache.hadoop.mapreduce.Job;
100 import org.apache.hadoop.mapreduce.JobContext;
101 import org.apache.hadoop.mapreduce.Mapper;
102 import org.apache.hadoop.mapreduce.RecordReader;
103 import org.apache.hadoop.mapreduce.Reducer;
104 import org.apache.hadoop.mapreduce.TaskAttemptContext;
105 import org.apache.hadoop.mapreduce.TaskAttemptID;
106 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
107 import org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat;
108 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
109 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
110 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
111 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
112 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
113 import org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat;
114 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
115 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
116 import org.apache.hadoop.util.Tool;
117 import org.apache.hadoop.util.ToolRunner;
118 import org.junit.Test;
119 import org.junit.experimental.categories.Category;
120
121 import com.google.common.collect.Sets;
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186 @Category(IntegrationTests.class)
187 public class IntegrationTestBigLinkedList extends IntegrationTestBase {
188 protected static final byte[] NO_KEY = new byte[1];
189
190 protected static String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
191
192 protected static String DEFAULT_TABLE_NAME = "IntegrationTestBigLinkedList";
193
194 protected static byte[] FAMILY_NAME = Bytes.toBytes("meta");
195
196
197 protected static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
198
199
200 protected static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
201
202
203 protected static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
204
205
206 private static final String GENERATOR_NUM_ROWS_PER_MAP_KEY
207 = "IntegrationTestBigLinkedList.generator.num_rows";
208
209 private static final String GENERATOR_NUM_MAPPERS_KEY
210 = "IntegrationTestBigLinkedList.generator.map.tasks";
211
212 private static final String GENERATOR_WIDTH_KEY
213 = "IntegrationTestBigLinkedList.generator.width";
214
215 private static final String GENERATOR_WRAP_KEY
216 = "IntegrationTestBigLinkedList.generator.wrap";
217
218 protected int NUM_SLAVES_BASE = 3;
219
220 private static final int MISSING_ROWS_TO_LOG = 50;
221
222 private static final int WIDTH_DEFAULT = 1000000;
223 private static final int WRAP_DEFAULT = 25;
224 private static final int ROWKEY_LENGTH = 16;
225
226 protected String toRun;
227 protected String[] otherArgs;
228
229 static class CINode {
230 byte[] key;
231 byte[] prev;
232 String client;
233 long count;
234 }
235
236
237
238
239 static class Generator extends Configured implements Tool {
240
241 private static final Log LOG = LogFactory.getLog(Generator.class);
242
243 static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
244 static class GeneratorInputSplit extends InputSplit implements Writable {
245 @Override
246 public long getLength() throws IOException, InterruptedException {
247 return 1;
248 }
249 @Override
250 public String[] getLocations() throws IOException, InterruptedException {
251 return new String[0];
252 }
253 @Override
254 public void readFields(DataInput arg0) throws IOException {
255 }
256 @Override
257 public void write(DataOutput arg0) throws IOException {
258 }
259 }
260
261 static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
262 private long count;
263 private long numNodes;
264 private Random rand;
265
266 @Override
267 public void close() throws IOException {
268 }
269
270 @Override
271 public BytesWritable getCurrentKey() throws IOException, InterruptedException {
272 byte[] bytes = new byte[ROWKEY_LENGTH];
273 rand.nextBytes(bytes);
274 return new BytesWritable(bytes);
275 }
276
277 @Override
278 public NullWritable getCurrentValue() throws IOException, InterruptedException {
279 return NullWritable.get();
280 }
281
282 @Override
283 public float getProgress() throws IOException, InterruptedException {
284 return (float)(count / (double)numNodes);
285 }
286
287 @Override
288 public void initialize(InputSplit arg0, TaskAttemptContext context)
289 throws IOException, InterruptedException {
290 numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
291 rand = new Random();
292 }
293
294 @Override
295 public boolean nextKeyValue() throws IOException, InterruptedException {
296 return count++ < numNodes;
297 }
298
299 }
300
301 @Override
302 public RecordReader<BytesWritable,NullWritable> createRecordReader(
303 InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
304 GeneratorRecordReader rr = new GeneratorRecordReader();
305 rr.initialize(split, context);
306 return rr;
307 }
308
309 @Override
310 public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
311 int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
312
313 ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
314
315 for (int i = 0; i < numMappers; i++) {
316 splits.add(new GeneratorInputSplit());
317 }
318
319 return splits;
320 }
321 }
322
323
324 static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
325 @Override
326 protected boolean isSplitable(JobContext context, Path filename) {
327 return false;
328 }
329 }
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355 static class GeneratorMapper
356 extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
357
358 byte[][] first = null;
359 byte[][] prev = null;
360 byte[][] current = null;
361 byte[] id;
362 long count = 0;
363 int i;
364 BufferedMutator mutator;
365 Connection connection;
366 long numNodes;
367 long wrap;
368 int width;
369
370 @Override
371 protected void setup(Context context) throws IOException, InterruptedException {
372 id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID());
373 Configuration conf = context.getConfiguration();
374 connection = ConnectionFactory.createConnection(conf);
375 instantiateHTable();
376 this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
377 current = new byte[this.width][];
378 int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
379 this.wrap = (long)wrapMultiplier * width;
380 this.numNodes = context.getConfiguration().getLong(
381 GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
382 if (this.numNodes < this.wrap) {
383 this.wrap = this.numNodes;
384 }
385 }
386
387 protected void instantiateHTable() throws IOException {
388 mutator = connection.getBufferedMutator(
389 new BufferedMutatorParams(getTableName(connection.getConfiguration()))
390 .writeBufferSize(4 * 1024 * 1024));
391 }
392
393 @Override
394 protected void cleanup(Context context) throws IOException ,InterruptedException {
395 mutator.close();
396 connection.close();
397 }
398
399 @Override
400 protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
401 current[i] = new byte[key.getLength()];
402 System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
403 if (++i == current.length) {
404 LOG.info("Persisting current.length=" + current.length + ", count=" + count + ", id=" +
405 Bytes.toStringBinary(id) + ", current=" + Bytes.toStringBinary(current[0]) +
406 ", i=" + i);
407 persist(output, count, prev, current, id);
408 i = 0;
409
410 if (first == null)
411 first = current;
412 prev = current;
413 current = new byte[this.width][];
414
415 count += current.length;
416 output.setStatus("Count " + count);
417
418 if (count % wrap == 0) {
419
420
421 circularLeftShift(first);
422
423 persist(output, -1, prev, first, null);
424
425 first = null;
426 prev = null;
427 }
428 }
429 }
430
431 private static <T> void circularLeftShift(T[] first) {
432 T ez = first[0];
433 System.arraycopy(first, 1, first, 0, first.length - 1);
434 first[first.length - 1] = ez;
435 }
436
437 protected void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
438 throws IOException {
439 for (int i = 0; i < current.length; i++) {
440 Put put = new Put(current[i]);
441 put.add(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
442
443 if (count >= 0) {
444 put.add(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
445 }
446 if (id != null) {
447 put.add(FAMILY_NAME, COLUMN_CLIENT, id);
448 }
449 mutator.mutate(put);
450
451 if (i % 1000 == 0) {
452
453 output.progress();
454 }
455 }
456
457 mutator.flush();
458 }
459 }
460
461 @Override
462 public int run(String[] args) throws Exception {
463 if (args.length < 3) {
464 System.out.println("Usage : " + Generator.class.getSimpleName() +
465 " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>]");
466 System.out.println(" where <num nodes per map> should be a multiple of " +
467 " width*wrap multiplier, 25M by default");
468 return 0;
469 }
470
471 int numMappers = Integer.parseInt(args[0]);
472 long numNodes = Long.parseLong(args[1]);
473 Path tmpOutput = new Path(args[2]);
474 Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
475 Integer wrapMuplitplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
476 return run(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
477 }
478
479 protected void createSchema() throws IOException {
480 Configuration conf = getConf();
481 Admin admin = new HBaseAdmin(conf);
482 TableName tableName = getTableName(conf);
483 try {
484 if (!admin.tableExists(tableName)) {
485 HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
486 htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
487 int numberOfServers = admin.getClusterStatus().getServers().size();
488 if (numberOfServers == 0) {
489 throw new IllegalStateException("No live regionservers");
490 }
491 int regionsPerServer = conf.getInt(HBaseTestingUtility.REGIONS_PER_SERVER_KEY,
492 HBaseTestingUtility.DEFAULT_REGIONS_PER_SERVER);
493 int totalNumberOfRegions = numberOfServers * regionsPerServer;
494 LOG.info("Number of live regionservers: " + numberOfServers + ", " +
495 "pre-splitting table into " + totalNumberOfRegions + " regions " +
496 "(default regions per server: " + regionsPerServer + ")");
497
498 byte[][] splits = new RegionSplitter.UniformSplit().split(totalNumberOfRegions);
499
500 admin.createTable(htd, splits);
501 }
502 } catch (MasterNotRunningException e) {
503 LOG.error("Master not running", e);
504 throw new IOException(e);
505 } finally {
506 admin.close();
507 }
508 }
509
510 public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
511 Integer width, Integer wrapMuplitplier) throws Exception {
512 LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
513 + ", numNodes=" + numNodes);
514 Job job = new Job(getConf());
515
516 job.setJobName("Random Input Generator");
517 job.setNumReduceTasks(0);
518 job.setJarByClass(getClass());
519
520 job.setInputFormatClass(GeneratorInputFormat.class);
521 job.setOutputKeyClass(BytesWritable.class);
522 job.setOutputValueClass(NullWritable.class);
523
524 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
525
526 job.setMapperClass(Mapper.class);
527
528 FileOutputFormat.setOutputPath(job, tmpOutput);
529 job.setOutputFormatClass(SequenceFileOutputFormat.class);
530
531 boolean success = jobCompletion(job);
532
533 return success ? 0 : 1;
534 }
535
536 public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
537 Integer width, Integer wrapMuplitplier) throws Exception {
538 LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
539 createSchema();
540 Job job = new Job(getConf());
541
542 job.setJobName("Link Generator");
543 job.setNumReduceTasks(0);
544 job.setJarByClass(getClass());
545
546 FileInputFormat.setInputPaths(job, tmpOutput);
547 job.setInputFormatClass(OneFilePerMapperSFIF.class);
548 job.setOutputKeyClass(NullWritable.class);
549 job.setOutputValueClass(NullWritable.class);
550
551 setJobConf(job, numMappers, numNodes, width, wrapMuplitplier);
552
553 setMapperForGenerator(job);
554
555 job.setOutputFormatClass(NullOutputFormat.class);
556
557 job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
558 TableMapReduceUtil.addDependencyJars(job);
559 TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
560 TableMapReduceUtil.initCredentials(job);
561
562 boolean success = jobCompletion(job);
563
564 return success ? 0 : 1;
565 }
566
567 protected boolean jobCompletion(Job job) throws IOException, InterruptedException,
568 ClassNotFoundException {
569 boolean success = job.waitForCompletion(true);
570 return success;
571 }
572
573 protected void setMapperForGenerator(Job job) {
574 job.setMapperClass(GeneratorMapper.class);
575 }
576
577 public int run(int numMappers, long numNodes, Path tmpOutput,
578 Integer width, Integer wrapMuplitplier) throws Exception {
579 int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
580 if (ret > 0) {
581 return ret;
582 }
583 return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMuplitplier);
584 }
585 }
586
587
588
589
590
591
592
593 static class Search extends Configured implements Tool {
594 private static final Log LOG = LogFactory.getLog(Search.class);
595 protected Job job;
596
597 private static void printUsage(final String error) {
598 if (error != null && error.length() > 0) System.out.println("ERROR: " + error);
599 System.err.println("Usage: search <KEYS_DIR> [<MAPPERS_COUNT>]");
600 }
601
602 @Override
603 public int run(String[] args) throws Exception {
604 if (args.length < 1 || args.length > 2) {
605 printUsage(null);
606 return 1;
607 }
608 Path inputDir = new Path(args[0]);
609 int numMappers = 1;
610 if (args.length > 1) {
611 numMappers = Integer.parseInt(args[1]);
612 }
613 return run(inputDir, numMappers);
614 }
615
616
617
618
619 public static class WALSearcher extends WALPlayer {
620 public WALSearcher(Configuration conf) {
621 super(conf);
622 }
623
624
625
626
627 public static class WALMapperSearcher extends WALMapper {
628 private SortedSet<byte []> keysToFind;
629
630 @Override
631 public void setup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation>.Context context)
632 throws IOException {
633 super.setup(context);
634 try {
635 this.keysToFind = readKeysToSearch(context.getConfiguration());
636 LOG.info("Loaded keys to find: count=" + this.keysToFind.size());
637 } catch (InterruptedException e) {
638 throw new InterruptedIOException(e.toString());
639 }
640 }
641
642 @Override
643 protected boolean filter(Context context, Cell cell) {
644
645 byte [] row = new byte [cell.getRowLength()];
646 System.arraycopy(cell.getRowArray(), cell.getRowOffset(), row, 0, cell.getRowLength());
647 boolean b = this.keysToFind.contains(row);
648 if (b) {
649 String keyStr = Bytes.toStringBinary(row);
650 LOG.info("Found cell=" + cell);
651 context.getCounter(FOUND_GROUP_KEY, keyStr).increment(1);
652 }
653 return b;
654 }
655 }
656
657
658 @Override
659 public Job createSubmittableJob(String[] args) throws IOException {
660 Job job = super.createSubmittableJob(args);
661
662 job.setJarByClass(WALMapperSearcher.class);
663 job.setMapperClass(WALMapperSearcher.class);
664 job.setOutputFormatClass(NullOutputFormat.class);
665 return job;
666 }
667 }
668
669 static final String FOUND_GROUP_KEY = "Found";
670 static final String SEARCHER_INPUTDIR_KEY = "searcher.keys.inputdir";
671
672 public int run(Path inputDir, int numMappers) throws Exception {
673 getConf().set(SEARCHER_INPUTDIR_KEY, inputDir.toString());
674 SortedSet<byte []> keys = readKeysToSearch(getConf());
675 if (keys.isEmpty()) throw new RuntimeException("No keys to find");
676 LOG.info("Count of keys to find: " + keys.size());
677 for(byte [] key: keys) LOG.info("Key: " + Bytes.toStringBinary(key));
678 Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
679
680 Path walsDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
681 Path oldWalsDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
682 LOG.info("Running Search with keys inputDir=" + inputDir +", numMappers=" + numMappers +
683 " against " + getConf().get(HConstants.HBASE_DIR));
684 int ret = ToolRunner.run(new WALSearcher(getConf()), new String [] {walsDir.toString(), ""});
685 if (ret != 0) return ret;
686 return ToolRunner.run(new WALSearcher(getConf()), new String [] {oldWalsDir.toString(), ""});
687 }
688
689 static SortedSet<byte []> readKeysToSearch(final Configuration conf)
690 throws IOException, InterruptedException {
691 Path keysInputDir = new Path(conf.get(SEARCHER_INPUTDIR_KEY));
692 FileSystem fs = FileSystem.get(conf);
693 SortedSet<byte []> result = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
694 if (!fs.exists(keysInputDir)) {
695 throw new FileNotFoundException(keysInputDir.toString());
696 }
697 if (!fs.isDirectory(keysInputDir)) {
698 throw new UnsupportedOperationException("TODO");
699 } else {
700 RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(keysInputDir, false);
701 while(iterator.hasNext()) {
702 LocatedFileStatus keyFileStatus = iterator.next();
703
704 if (keyFileStatus.getPath().getName().startsWith("_")) continue;
705 result.addAll(readFileToSearch(conf, fs, keyFileStatus));
706 }
707 }
708 return result;
709 }
710
711 private static SortedSet<byte []> readFileToSearch(final Configuration conf,
712 final FileSystem fs, final LocatedFileStatus keyFileStatus)
713 throws IOException, InterruptedException {
714 SortedSet<byte []> result = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
715
716
717 TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
718 try (SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader rr =
719 new SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader()) {
720 InputSplit is =
721 new FileSplit(keyFileStatus.getPath(), 0, keyFileStatus.getLen(), new String [] {});
722 rr.initialize(is, context);
723 while (rr.nextKeyValue()) {
724 rr.getCurrentKey();
725 BytesWritable bw = rr.getCurrentValue();
726 switch (Verify.VerifyReducer.whichType(bw.getBytes())) {
727 case UNDEFINED:
728 byte [] key = new byte [rr.getCurrentKey().getLength()];
729 System.arraycopy(rr.getCurrentKey().getBytes(), 0, key, 0,
730 rr.getCurrentKey().getLength());
731 result.add(key);
732 break;
733 }
734 }
735 }
736 return result;
737 }
738 }
739
740
741
742
743
744 static class Verify extends Configured implements Tool {
745
746 private static final Log LOG = LogFactory.getLog(Verify.class);
747 protected static final BytesWritable DEF = new BytesWritable(NO_KEY);
748
749 protected Job job;
750
751 public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
752 private BytesWritable row = new BytesWritable();
753 private BytesWritable ref = new BytesWritable();
754
755 @Override
756 protected void map(ImmutableBytesWritable key, Result value, Context context)
757 throws IOException ,InterruptedException {
758 byte[] rowKey = key.get();
759 row.set(rowKey, 0, rowKey.length);
760 context.write(row, DEF);
761 byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
762 if (prev != null && prev.length > 0) {
763 ref.set(prev, 0, prev.length);
764 context.write(ref, row);
765 } else {
766 LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
767 }
768 }
769 }
770
771
772
773
774
775 public static enum Counts {
776 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES, EXTRA_UNDEF_REFERENCES
777 }
778
779
780
781
782
783
784 public static class VerifyReducer
785 extends Reducer<BytesWritable,BytesWritable,BytesWritable,BytesWritable> {
786 private ArrayList<byte[]> refs = new ArrayList<byte[]>();
787 private final BytesWritable UNREF =
788 new BytesWritable(addPrefixFlag(Counts.UNREFERENCED.ordinal(), new byte [] {}));
789
790 private AtomicInteger rows = new AtomicInteger(0);
791 private Connection connection;
792
793 @Override
794 protected void setup(Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
795 throws IOException, InterruptedException {
796 super.setup(context);
797 this.connection = ConnectionFactory.createConnection(context.getConfiguration());
798 }
799
800 @Override
801 protected void cleanup(Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
802 throws IOException, InterruptedException {
803 if (this.connection != null) this.connection.close();
804 super.cleanup(context);
805 }
806
807
808
809
810
811
812
813 public static byte [] addPrefixFlag(final int ordinal, final byte [] r) {
814 byte [] prefix = Bytes.toBytes((short)ordinal);
815 if (prefix.length != Bytes.SIZEOF_SHORT) {
816 throw new RuntimeException("Unexpected size: " + prefix.length);
817 }
818 byte [] result = new byte [prefix.length + r.length];
819 System.arraycopy(prefix, 0, result, 0, prefix.length);
820 System.arraycopy(r, 0, result, prefix.length, r.length);
821 return result;
822 }
823
824
825
826
827
828
829 public static Counts whichType(final byte [] bs) {
830 int ordinal = Bytes.toShort(bs, 0, Bytes.SIZEOF_SHORT);
831 return Counts.values()[ordinal];
832 }
833
834
835
836
837
838 public static byte [] getRowOnly(BytesWritable bw) {
839 byte [] bytes = new byte [bw.getLength() - Bytes.SIZEOF_SHORT];
840 System.arraycopy(bw.getBytes(), Bytes.SIZEOF_SHORT, bytes, 0, bytes.length);
841 return bytes;
842 }
843
844 @Override
845 public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
846 throws IOException, InterruptedException {
847
848 int defCount = 0;
849 refs.clear();
850 for (BytesWritable type : values) {
851 if (type.getLength() == DEF.getLength()) {
852 defCount++;
853 } else {
854 byte[] bytes = new byte[type.getLength()];
855 System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
856 refs.add(bytes);
857 }
858 }
859
860
861 StringBuilder refsSb = null;
862 String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
863 if (defCount == 0 || refs.size() != 1) {
864 refsSb = dumpExtraInfoOnRefs(key, context, refs);
865 LOG.error("LinkedListError: key=" + keyString + ", reference(s)=" +
866 (refsSb != null? refsSb.toString(): ""));
867 }
868
869 if (defCount == 0 && refs.size() > 0) {
870
871
872
873 for (int i = 0; i < refs.size(); i++) {
874 byte [] bs = refs.get(i);
875 int ordinal;
876 if (i <= 0) {
877 ordinal = Counts.UNDEFINED.ordinal();
878 context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
879 context.getCounter(Counts.UNDEFINED).increment(1);
880 } else {
881 ordinal = Counts.EXTRA_UNDEF_REFERENCES.ordinal();
882 context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
883 }
884 }
885 if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
886
887
888
889
890 context.getCounter("undef", keyString).increment(1);
891 }
892 } else if (defCount > 0 && refs.size() == 0) {
893
894 context.write(key, UNREF);
895 context.getCounter(Counts.UNREFERENCED).increment(1);
896 if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
897 context.getCounter("unref", keyString).increment(1);
898 }
899 } else {
900 if (refs.size() > 1) {
901
902 for (int i = 1; i < refs.size(); i++) {
903 context.write(key,
904 new BytesWritable(addPrefixFlag(Counts.EXTRAREFERENCES.ordinal(), refs.get(i))));
905 }
906 context.getCounter(Counts.EXTRAREFERENCES).increment(refs.size() - 1);
907 }
908
909 context.getCounter(Counts.REFERENCED).increment(1);
910 }
911 }
912
913
914
915
916
917
918 private StringBuilder dumpExtraInfoOnRefs(final BytesWritable key, final Context context,
919 final List<byte []> refs)
920 throws IOException {
921 StringBuilder refsSb = null;
922 if (refs.isEmpty()) return refsSb;
923 refsSb = new StringBuilder();
924 String comma = "";
925
926
927
928 TableName tn = getTableName(context.getConfiguration());
929 try (Table t = this.connection.getTable(tn)) {
930 for (byte [] ref : refs) {
931 Result r = t.get(new Get(ref));
932 List<Cell> cells = r.listCells();
933 String ts = (cells != null && !cells.isEmpty())?
934 new java.util.Date(cells.get(0).getTimestamp()).toString(): "";
935 byte [] b = r.getValue(FAMILY_NAME, COLUMN_CLIENT);
936 String jobStr = (b != null && b.length > 0)? Bytes.toString(b): "";
937 b = r.getValue(FAMILY_NAME, COLUMN_COUNT);
938 long count = (b != null && b.length > 0)? Bytes.toLong(b): -1;
939 b = r.getValue(FAMILY_NAME, COLUMN_PREV);
940 String refRegionLocation = "";
941 String keyRegionLocation = "";
942 if (b != null && b.length > 0) {
943 try (RegionLocator rl = this.connection.getRegionLocator(tn)) {
944 HRegionLocation hrl = rl.getRegionLocation(b);
945 if (hrl != null) refRegionLocation = hrl.toString();
946
947 hrl = rl.getRegionLocation(key.getBytes());
948 if (hrl != null) keyRegionLocation = hrl.toString();
949 }
950 }
951 LOG.error("Extras on ref without a def, ref=" + Bytes.toStringBinary(ref) +
952 ", refPrevEqualsKey=" +
953 (Bytes.compareTo(key.getBytes(), 0, key.getLength(), b, 0, b.length) == 0) +
954 ", key=" + Bytes.toStringBinary(key.getBytes(), 0, key.getLength()) +
955 ", ref row date=" + ts + ", jobStr=" + jobStr +
956 ", ref row count=" + count +
957 ", ref row regionLocation=" + refRegionLocation +
958 ", key row regionLocation=" + keyRegionLocation);
959 refsSb.append(comma);
960 comma = ",";
961 refsSb.append(Bytes.toStringBinary(ref));
962 }
963 }
964 return refsSb;
965 }
966 }
967
968 @Override
969 public int run(String[] args) throws Exception {
970
971 if (args.length != 2) {
972 System.out.println("Usage : " + Verify.class.getSimpleName() + " <output dir> <num reducers>");
973 return 0;
974 }
975
976 String outputDir = args[0];
977 int numReducers = Integer.parseInt(args[1]);
978
979 return run(outputDir, numReducers);
980 }
981
982 public int run(String outputDir, int numReducers) throws Exception {
983 return run(new Path(outputDir), numReducers);
984 }
985
986 public int run(Path outputDir, int numReducers) throws Exception {
987 LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
988
989 job = new Job(getConf());
990
991 job.setJobName("Link Verifier");
992 job.setNumReduceTasks(numReducers);
993 job.setJarByClass(getClass());
994
995 setJobScannerConf(job);
996
997 Scan scan = new Scan();
998 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
999 scan.setCaching(10000);
1000 scan.setCacheBlocks(false);
1001
1002 TableMapReduceUtil.initTableMapperJob(getTableName(getConf()).getName(), scan,
1003 VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
1004 TableMapReduceUtil.addDependencyJars(job.getConfiguration(), AbstractHBaseTool.class);
1005
1006 job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
1007
1008 job.setReducerClass(VerifyReducer.class);
1009 job.setOutputFormatClass(SequenceFileAsBinaryOutputFormat.class);
1010 job.setOutputKeyClass(BytesWritable.class);
1011 job.setOutputValueClass(BytesWritable.class);
1012 TextOutputFormat.setOutputPath(job, outputDir);
1013
1014 boolean success = job.waitForCompletion(true);
1015
1016 return success ? 0 : 1;
1017 }
1018
1019 @SuppressWarnings("deprecation")
1020 public boolean verify(long expectedReferenced) throws Exception {
1021 if (job == null) {
1022 throw new IllegalStateException("You should call run() first");
1023 }
1024
1025 Counters counters = job.getCounters();
1026
1027 Counter referenced = counters.findCounter(Counts.REFERENCED);
1028 Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
1029 Counter undefined = counters.findCounter(Counts.UNDEFINED);
1030 Counter multiref = counters.findCounter(Counts.EXTRAREFERENCES);
1031
1032 boolean success = true;
1033
1034 if (expectedReferenced != referenced.getValue()) {
1035 LOG.error("Expected referenced count does not match with actual referenced count. " +
1036 "expected referenced=" + expectedReferenced + " ,actual=" + referenced.getValue());
1037 success = false;
1038 }
1039
1040 if (unreferenced.getValue() > 0) {
1041 boolean couldBeMultiRef = (multiref.getValue() == unreferenced.getValue());
1042 LOG.error("Unreferenced nodes were not expected. Unreferenced count=" + unreferenced.getValue()
1043 + (couldBeMultiRef ? "; could be due to duplicate random numbers" : ""));
1044 success = false;
1045 }
1046
1047 if (undefined.getValue() > 0) {
1048 LOG.error("Found an undefined node. Undefined count=" + undefined.getValue());
1049 success = false;
1050 }
1051
1052 if (!success) {
1053 handleFailure(counters);
1054 }
1055 return success;
1056 }
1057
1058 protected void handleFailure(Counters counters) throws IOException {
1059 Configuration conf = job.getConfiguration();
1060 TableName tableName = getTableName(conf);
1061 try (Connection conn = ConnectionFactory.createConnection(conf)) {
1062 try (RegionLocator rl = conn.getRegionLocator(tableName)) {
1063 CounterGroup g = counters.getGroup("undef");
1064 Iterator<Counter> it = g.iterator();
1065 while (it.hasNext()) {
1066 String keyString = it.next().getName();
1067 byte[] key = Bytes.toBytes(keyString);
1068 HRegionLocation loc = rl.getRegionLocation(key, true);
1069 LOG.error("undefined row " + keyString + ", " + loc);
1070 }
1071 g = counters.getGroup("unref");
1072 it = g.iterator();
1073 while (it.hasNext()) {
1074 String keyString = it.next().getName();
1075 byte[] key = Bytes.toBytes(keyString);
1076 HRegionLocation loc = rl.getRegionLocation(key, true);
1077 LOG.error("unreferred row " + keyString + ", " + loc);
1078 }
1079 }
1080 }
1081 }
1082 }
1083
1084
1085
1086
1087
1088 static class Loop extends Configured implements Tool {
1089
1090 private static final Log LOG = LogFactory.getLog(Loop.class);
1091
1092 IntegrationTestBigLinkedList it;
1093
1094 protected void runGenerator(int numMappers, long numNodes,
1095 String outputDir, Integer width, Integer wrapMuplitplier) throws Exception {
1096 Path outputPath = new Path(outputDir);
1097 UUID uuid = UUID.randomUUID();
1098 Path generatorOutput = new Path(outputPath, uuid.toString());
1099
1100 Generator generator = new Generator();
1101 generator.setConf(getConf());
1102 int retCode = generator.run(numMappers, numNodes, generatorOutput, width, wrapMuplitplier);
1103 if (retCode > 0) {
1104 throw new RuntimeException("Generator failed with return code: " + retCode);
1105 }
1106 }
1107
1108 protected void runVerify(String outputDir,
1109 int numReducers, long expectedNumNodes) throws Exception {
1110 Path outputPath = new Path(outputDir);
1111 UUID uuid = UUID.randomUUID();
1112 Path iterationOutput = new Path(outputPath, uuid.toString());
1113
1114 Verify verify = new Verify();
1115 verify.setConf(getConf());
1116 int retCode = verify.run(iterationOutput, numReducers);
1117 if (retCode > 0) {
1118 throw new RuntimeException("Verify.run failed with return code: " + retCode);
1119 }
1120
1121 if (!verify.verify(expectedNumNodes)) {
1122 throw new RuntimeException("Verify.verify failed");
1123 }
1124
1125 LOG.info("Verify finished with succees. Total nodes=" + expectedNumNodes);
1126 }
1127
1128 @Override
1129 public int run(String[] args) throws Exception {
1130 if (args.length < 5) {
1131 System.err.println("Usage: Loop <num iterations> <num mappers> <num nodes per mapper> <output dir> <num reducers> [<width> <wrap multiplier>]");
1132 return 1;
1133 }
1134 LOG.info("Running Loop with args:" + Arrays.deepToString(args));
1135
1136 int numIterations = Integer.parseInt(args[0]);
1137 int numMappers = Integer.parseInt(args[1]);
1138 long numNodes = Long.parseLong(args[2]);
1139 String outputDir = args[3];
1140 int numReducers = Integer.parseInt(args[4]);
1141 Integer width = (args.length < 6) ? null : Integer.parseInt(args[5]);
1142 Integer wrapMuplitplier = (args.length < 7) ? null : Integer.parseInt(args[6]);
1143
1144 long expectedNumNodes = 0;
1145
1146 if (numIterations < 0) {
1147 numIterations = Integer.MAX_VALUE;
1148 }
1149
1150 for (int i = 0; i < numIterations; i++) {
1151 LOG.info("Starting iteration = " + i);
1152 runGenerator(numMappers, numNodes, outputDir, width, wrapMuplitplier);
1153 expectedNumNodes += numMappers * numNodes;
1154
1155 runVerify(outputDir, numReducers, expectedNumNodes);
1156 }
1157
1158 return 0;
1159 }
1160 }
1161
1162
1163
1164
1165 private static class Print extends Configured implements Tool {
1166 @Override
1167 public int run(String[] args) throws Exception {
1168 Options options = new Options();
1169 options.addOption("s", "start", true, "start key");
1170 options.addOption("e", "end", true, "end key");
1171 options.addOption("l", "limit", true, "number to print");
1172
1173 GnuParser parser = new GnuParser();
1174 CommandLine cmd = null;
1175 try {
1176 cmd = parser.parse(options, args);
1177 if (cmd.getArgs().length != 0) {
1178 throw new ParseException("Command takes no arguments");
1179 }
1180 } catch (ParseException e) {
1181 System.err.println("Failed to parse command line " + e.getMessage());
1182 System.err.println();
1183 HelpFormatter formatter = new HelpFormatter();
1184 formatter.printHelp(getClass().getSimpleName(), options);
1185 System.exit(-1);
1186 }
1187
1188 Table table = new HTable(getConf(), getTableName(getConf()));
1189
1190 Scan scan = new Scan();
1191 scan.setBatch(10000);
1192
1193 if (cmd.hasOption("s"))
1194 scan.setStartRow(Bytes.toBytesBinary(cmd.getOptionValue("s")));
1195
1196 if (cmd.hasOption("e"))
1197 scan.setStopRow(Bytes.toBytesBinary(cmd.getOptionValue("e")));
1198
1199 int limit = 0;
1200 if (cmd.hasOption("l"))
1201 limit = Integer.parseInt(cmd.getOptionValue("l"));
1202 else
1203 limit = 100;
1204
1205 ResultScanner scanner = table.getScanner(scan);
1206
1207 CINode node = new CINode();
1208 Result result = scanner.next();
1209 int count = 0;
1210 while (result != null && count++ < limit) {
1211 node = getCINode(result, node);
1212 System.out.printf("%s:%s:%012d:%s\n", Bytes.toStringBinary(node.key),
1213 Bytes.toStringBinary(node.prev), node.count, node.client);
1214 result = scanner.next();
1215 }
1216 scanner.close();
1217 table.close();
1218
1219 return 0;
1220 }
1221 }
1222
1223
1224
1225
1226 private static class Delete extends Configured implements Tool {
1227 @Override
1228 public int run(String[] args) throws Exception {
1229 if (args.length != 1) {
1230 System.out.println("Usage : " + Delete.class.getSimpleName() + " <node to delete>");
1231 return 0;
1232 }
1233 byte[] val = Bytes.toBytesBinary(args[0]);
1234
1235 org.apache.hadoop.hbase.client.Delete delete
1236 = new org.apache.hadoop.hbase.client.Delete(val);
1237
1238 Table table = new HTable(getConf(), getTableName(getConf()));
1239 table.delete(delete);
1240 table.close();
1241
1242 System.out.println("Delete successful");
1243 return 0;
1244 }
1245 }
1246
1247
1248
1249
1250
1251 private static class Walker extends Configured implements Tool {
1252 @Override
1253 public int run(String[] args) throws IOException {
1254 Options options = new Options();
1255 options.addOption("n", "num", true, "number of queries");
1256 options.addOption("s", "start", true, "key to start at, binary string");
1257 options.addOption("l", "logevery", true, "log every N queries");
1258
1259 GnuParser parser = new GnuParser();
1260 CommandLine cmd = null;
1261 try {
1262 cmd = parser.parse(options, args);
1263 if (cmd.getArgs().length != 0) {
1264 throw new ParseException("Command takes no arguments");
1265 }
1266 } catch (ParseException e) {
1267 System.err.println("Failed to parse command line " + e.getMessage());
1268 System.err.println();
1269 HelpFormatter formatter = new HelpFormatter();
1270 formatter.printHelp(getClass().getSimpleName(), options);
1271 System.exit(-1);
1272 }
1273
1274 long maxQueries = Long.MAX_VALUE;
1275 if (cmd.hasOption('n')) {
1276 maxQueries = Long.parseLong(cmd.getOptionValue("n"));
1277 }
1278 Random rand = new Random();
1279 boolean isSpecificStart = cmd.hasOption('s');
1280 byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
1281 int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1;
1282
1283 Table table = new HTable(getConf(), getTableName(getConf()));
1284 long numQueries = 0;
1285
1286
1287
1288 while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) {
1289 if (!isSpecificStart) {
1290 startKey = new byte[ROWKEY_LENGTH];
1291 rand.nextBytes(startKey);
1292 }
1293 CINode node = findStartNode(table, startKey);
1294 if (node == null && isSpecificStart) {
1295 System.err.printf("Start node not found: %s \n", Bytes.toStringBinary(startKey));
1296 }
1297 numQueries++;
1298 while (node != null && node.prev.length != NO_KEY.length && numQueries < maxQueries) {
1299 byte[] prev = node.prev;
1300 long t1 = System.currentTimeMillis();
1301 node = getNode(prev, table, node);
1302 long t2 = System.currentTimeMillis();
1303 if (numQueries % logEvery == 0) {
1304 System.out.printf("CQ %d: %d %s \n", numQueries, t2 - t1, Bytes.toStringBinary(prev));
1305 }
1306 numQueries++;
1307 if (node == null) {
1308 System.err.printf("UNDEFINED NODE %s \n", Bytes.toStringBinary(prev));
1309 } else if (node.prev.length == NO_KEY.length) {
1310 System.err.printf("TERMINATING NODE %s \n", Bytes.toStringBinary(node.key));
1311 }
1312 }
1313 }
1314
1315 table.close();
1316 return 0;
1317 }
1318
1319 private static CINode findStartNode(Table table, byte[] startKey) throws IOException {
1320 Scan scan = new Scan();
1321 scan.setStartRow(startKey);
1322 scan.setBatch(1);
1323 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
1324
1325 long t1 = System.currentTimeMillis();
1326 ResultScanner scanner = table.getScanner(scan);
1327 Result result = scanner.next();
1328 long t2 = System.currentTimeMillis();
1329 scanner.close();
1330
1331 if ( result != null) {
1332 CINode node = getCINode(result, new CINode());
1333 System.out.printf("FSR %d %s\n", t2 - t1, Bytes.toStringBinary(node.key));
1334 return node;
1335 }
1336
1337 System.out.println("FSR " + (t2 - t1));
1338
1339 return null;
1340 }
1341
1342 private CINode getNode(byte[] row, Table table, CINode node) throws IOException {
1343 Get get = new Get(row);
1344 get.addColumn(FAMILY_NAME, COLUMN_PREV);
1345 Result result = table.get(get);
1346 return getCINode(result, node);
1347 }
1348 }
1349
1350 private static class Clean extends Configured implements Tool {
1351 @Override public int run(String[] args) throws Exception {
1352 if (args.length < 1) {
1353 System.err.println("Usage: Clean <output dir>");
1354 return -1;
1355 }
1356
1357 Path p = new Path(args[0]);
1358 Configuration conf = getConf();
1359 TableName tableName = getTableName(conf);
1360
1361 FileSystem fs = HFileSystem.get(conf);
1362 Admin admin = new HBaseAdmin(conf);
1363
1364 if (admin.tableExists(tableName)) {
1365 admin.disableTable(tableName);
1366 admin.deleteTable(tableName);
1367 }
1368
1369 if (fs.exists(p)) {
1370 fs.delete(p, true);
1371 }
1372
1373 return 0;
1374 }
1375 }
1376
1377 static TableName getTableName(Configuration conf) {
1378 return TableName.valueOf(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1379 }
1380
1381 private static CINode getCINode(Result result, CINode node) {
1382 node.key = Bytes.copy(result.getRow());
1383 if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
1384 node.prev = Bytes.copy(result.getValue(FAMILY_NAME, COLUMN_PREV));
1385 } else {
1386 node.prev = NO_KEY;
1387 }
1388 if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
1389 node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
1390 } else {
1391 node.count = -1;
1392 }
1393 if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
1394 node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
1395 } else {
1396 node.client = "";
1397 }
1398 return node;
1399 }
1400
1401 protected IntegrationTestingUtility util;
1402
1403 @Override
1404 public void setUpCluster() throws Exception {
1405 util = getTestingUtil(getConf());
1406 boolean isDistributed = util.isDistributedCluster();
1407 util.initializeCluster(isDistributed ? 1 : this.NUM_SLAVES_BASE);
1408 if (!isDistributed) {
1409 util.startMiniMapReduceCluster();
1410 }
1411 this.setConf(util.getConfiguration());
1412 }
1413
1414 @Override
1415 public void cleanUpCluster() throws Exception {
1416 super.cleanUpCluster();
1417 if (util.isDistributedCluster()) {
1418 util.shutdownMiniMapReduceCluster();
1419 }
1420 }
1421
1422 @Test
1423 public void testContinuousIngest() throws IOException, Exception {
1424
1425 int ret = ToolRunner.run(getTestingUtil(getConf()).getConfiguration(), new Loop(),
1426 new String[] {"1", "1", "2000000",
1427 util.getDataTestDirOnTestFS("IntegrationTestBigLinkedList").toString(), "1"});
1428 org.junit.Assert.assertEquals(0, ret);
1429 }
1430
1431 private void usage() {
1432 System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1433 printCommands();
1434 }
1435
1436 private void printCommands() {
1437 System.err.println("Commands:");
1438 System.err.println(" generator Map only job that generates data.");
1439 System.err.println(" verify A map reduce job that looks for holes. Look at the counts ");
1440 System.err.println(" after running. See REFERENCED and UNREFERENCED are ok. Any ");
1441 System.err.println(" UNDEFINED counts are bad. Do not run with the Generator.");
1442 System.err.println(" walker " +
1443 "Standalone program that starts following a linked list & emits timing info.");
1444 System.err.println(" print Standalone program that prints nodes in the linked list.");
1445 System.err.println(" delete Standalone program that deletes a·single node.");
1446 System.err.println(" loop Program to Loop through Generator and Verify steps");
1447 System.err.println(" clean Program to clean all left over detritus.");
1448 System.err.println(" search Search for missing keys.");
1449 System.err.flush();
1450 }
1451
1452 @Override
1453 protected void processOptions(CommandLine cmd) {
1454 super.processOptions(cmd);
1455 String[] args = cmd.getArgs();
1456
1457 if (args.length < 1) {
1458 printUsage(this.getClass().getSimpleName() +
1459 " <general options> COMMAND [<COMMAND options>]", "General options:", "");
1460 printCommands();
1461
1462 throw new RuntimeException("Incorrect Number of args.");
1463 }
1464 toRun = args[0];
1465 otherArgs = Arrays.copyOfRange(args, 1, args.length);
1466 }
1467
1468 @Override
1469 public int runTestFromCommandLine() throws Exception {
1470
1471 Tool tool = null;
1472 if (toRun.equalsIgnoreCase("Generator")) {
1473 tool = new Generator();
1474 } else if (toRun.equalsIgnoreCase("Verify")) {
1475 tool = new Verify();
1476 } else if (toRun.equalsIgnoreCase("Loop")) {
1477 Loop loop = new Loop();
1478 loop.it = this;
1479 tool = loop;
1480 } else if (toRun.equalsIgnoreCase("Walker")) {
1481 tool = new Walker();
1482 } else if (toRun.equalsIgnoreCase("Print")) {
1483 tool = new Print();
1484 } else if (toRun.equalsIgnoreCase("Delete")) {
1485 tool = new Delete();
1486 } else if (toRun.equalsIgnoreCase("Clean")) {
1487 tool = new Clean();
1488 } else if (toRun.equalsIgnoreCase("Search")) {
1489 tool = new Search();
1490 } else {
1491 usage();
1492 throw new RuntimeException("Unknown arg");
1493 }
1494
1495 return ToolRunner.run(getConf(), tool, otherArgs);
1496 }
1497
1498 @Override
1499 public TableName getTablename() {
1500 Configuration c = getConf();
1501 return TableName.valueOf(c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1502 }
1503
1504 @Override
1505 protected Set<String> getColumnFamilies() {
1506 return Sets.newHashSet(Bytes.toString(FAMILY_NAME));
1507 }
1508
1509 private static void setJobConf(Job job, int numMappers, long numNodes,
1510 Integer width, Integer wrapMultiplier) {
1511 job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1512 job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1513 if (width != null) {
1514 job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width);
1515 }
1516 if (wrapMultiplier != null) {
1517 job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMultiplier);
1518 }
1519 }
1520
1521 public static void setJobScannerConf(Job job) {
1522
1523 job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1524 job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1525 }
1526
1527 public static void main(String[] args) throws Exception {
1528 Configuration conf = HBaseConfiguration.create();
1529 IntegrationTestingUtility.setUseDistributedCluster(conf);
1530 int ret = ToolRunner.run(conf, new IntegrationTestBigLinkedList(), args);
1531 System.exit(ret);
1532 }
1533 }