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