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 com.google.common.base.Preconditions;
22 import com.google.common.collect.Sets;
23 import java.io.DataInput;
24 import java.io.DataOutput;
25 import java.io.FileNotFoundException;
26 import java.io.IOException;
27 import java.io.InterruptedIOException;
28 import java.security.SecureRandom;
29 import java.util.ArrayList;
30 import java.util.Arrays;
31 import java.util.Iterator;
32 import java.util.List;
33 import java.util.Random;
34 import java.util.Set;
35 import java.util.SortedSet;
36 import java.util.TreeSet;
37 import java.util.UUID;
38 import java.util.concurrent.ThreadLocalRandom;
39 import java.util.concurrent.atomic.AtomicInteger;
40 import org.apache.commons.cli.CommandLine;
41 import org.apache.commons.cli.GnuParser;
42 import org.apache.commons.cli.HelpFormatter;
43 import org.apache.commons.cli.Options;
44 import org.apache.commons.cli.ParseException;
45 import org.apache.commons.logging.Log;
46 import org.apache.commons.logging.LogFactory;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.conf.Configured;
49 import org.apache.hadoop.fs.FileSystem;
50 import org.apache.hadoop.fs.LocatedFileStatus;
51 import org.apache.hadoop.fs.Path;
52 import org.apache.hadoop.fs.RemoteIterator;
53 import org.apache.hadoop.hbase.Cell;
54 import org.apache.hadoop.hbase.HBaseConfiguration;
55 import org.apache.hadoop.hbase.HBaseTestingUtility;
56 import org.apache.hadoop.hbase.HColumnDescriptor;
57 import org.apache.hadoop.hbase.HConstants;
58 import org.apache.hadoop.hbase.HRegionLocation;
59 import org.apache.hadoop.hbase.HTableDescriptor;
60 import org.apache.hadoop.hbase.IntegrationTestBase;
61 import org.apache.hadoop.hbase.IntegrationTestingUtility;
62 import org.apache.hadoop.hbase.MasterNotRunningException;
63 import org.apache.hadoop.hbase.TableName;
64 import org.apache.hadoop.hbase.client.Admin;
65 import org.apache.hadoop.hbase.client.BufferedMutator;
66 import org.apache.hadoop.hbase.client.BufferedMutatorParams;
67 import org.apache.hadoop.hbase.client.Connection;
68 import org.apache.hadoop.hbase.client.ConnectionConfiguration;
69 import org.apache.hadoop.hbase.client.ConnectionFactory;
70 import org.apache.hadoop.hbase.client.Get;
71 import org.apache.hadoop.hbase.client.HTable;
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.fs.HFileSystem;
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.FlushLargeStoresPolicy;
87 import org.apache.hadoop.hbase.regionserver.FlushPolicyFactory;
88 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
89 import org.apache.hadoop.hbase.testclassification.IntegrationTests;
90 import org.apache.hadoop.hbase.util.AbstractHBaseTool;
91 import org.apache.hadoop.hbase.util.Bytes;
92 import org.apache.hadoop.hbase.util.Random64;
93 import org.apache.hadoop.hbase.util.RegionSplitter;
94 import org.apache.hadoop.hbase.wal.WALKey;
95 import org.apache.hadoop.io.BytesWritable;
96 import org.apache.hadoop.io.NullWritable;
97 import org.apache.hadoop.io.Writable;
98 import org.apache.hadoop.mapreduce.Counter;
99 import org.apache.hadoop.mapreduce.CounterGroup;
100 import org.apache.hadoop.mapreduce.Counters;
101 import org.apache.hadoop.mapreduce.InputFormat;
102 import org.apache.hadoop.mapreduce.InputSplit;
103 import org.apache.hadoop.mapreduce.Job;
104 import org.apache.hadoop.mapreduce.JobContext;
105 import org.apache.hadoop.mapreduce.Mapper;
106 import org.apache.hadoop.mapreduce.RecordReader;
107 import org.apache.hadoop.mapreduce.Reducer;
108 import org.apache.hadoop.mapreduce.TaskAttemptContext;
109 import org.apache.hadoop.mapreduce.TaskAttemptID;
110 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
111 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
112 import org.apache.hadoop.mapreduce.lib.input.SequenceFileAsBinaryInputFormat;
113 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
114 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
115 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
116 import org.apache.hadoop.mapreduce.lib.output.SequenceFileAsBinaryOutputFormat;
117 import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
118 import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
119 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
120 import org.apache.hadoop.util.Tool;
121 import org.apache.hadoop.util.ToolRunner;
122 import org.junit.Test;
123 import org.junit.experimental.categories.Category;
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
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224 @Category(IntegrationTests.class)
225 public class IntegrationTestBigLinkedList extends IntegrationTestBase {
226 protected static final byte[] NO_KEY = new byte[1];
227
228 protected static String TABLE_NAME_KEY = "IntegrationTestBigLinkedList.table";
229
230 protected static String DEFAULT_TABLE_NAME = "IntegrationTestBigLinkedList";
231
232 protected static byte[] FAMILY_NAME = Bytes.toBytes("meta");
233 private static byte[] BIG_FAMILY_NAME = Bytes.toBytes("big");
234 private static byte[] TINY_FAMILY_NAME = Bytes.toBytes("tiny");
235
236
237 protected static final byte[] COLUMN_PREV = Bytes.toBytes("prev");
238
239
240 protected static final byte[] COLUMN_CLIENT = Bytes.toBytes("client");
241
242
243 protected static final byte[] COLUMN_COUNT = Bytes.toBytes("count");
244
245
246 private static final String GENERATOR_NUM_ROWS_PER_MAP_KEY
247 = "IntegrationTestBigLinkedList.generator.num_rows";
248
249 private static final String GENERATOR_NUM_MAPPERS_KEY
250 = "IntegrationTestBigLinkedList.generator.map.tasks";
251
252 private static final String GENERATOR_WIDTH_KEY
253 = "IntegrationTestBigLinkedList.generator.width";
254
255 private static final String GENERATOR_WRAP_KEY
256 = "IntegrationTestBigLinkedList.generator.wrap";
257
258 private static final String CONCURRENT_WALKER_KEY
259 = "IntegrationTestBigLinkedList.generator.concurrentwalkers";
260
261 protected int NUM_SLAVES_BASE = 3;
262
263 private static final int MISSING_ROWS_TO_LOG = 10;
264
265 private static final int WIDTH_DEFAULT = 1000000;
266 private static final int WRAP_DEFAULT = 25;
267 private static final int ROWKEY_LENGTH = 16;
268
269 private static final int CONCURRENT_WALKER_DEFAULT = 0;
270
271 protected String toRun;
272 protected String[] otherArgs;
273
274 static class CINode {
275 byte[] key;
276 byte[] prev;
277 String client;
278 long count;
279 }
280
281
282
283
284 static class Generator extends Configured implements Tool {
285
286 private static final Log LOG = LogFactory.getLog(Generator.class);
287
288
289
290
291
292
293
294
295
296
297
298
299 public static final String MULTIPLE_UNEVEN_COLUMNFAMILIES_KEY =
300 "generator.multiple.columnfamilies";
301
302
303
304
305
306
307
308 public static final String BIG_FAMILY_VALUE_SIZE_KEY = "generator.big.family.value.size";
309
310
311 public static enum Counts {
312 SUCCESS, TERMINATING, UNDEFINED, IOEXCEPTION
313 }
314
315 public static final String USAGE = "Usage : " + Generator.class.getSimpleName() +
316 " <num mappers> <num nodes per map> <tmp output dir> [<width> <wrap multiplier>" +
317 " <num walker threads>] \n" +
318 "where <num nodes per map> should be a multiple of width*wrap multiplier, 25M by default \n" +
319 "walkers will verify random flushed loop during Generation.";
320
321 public Job job;
322
323 static class GeneratorInputFormat extends InputFormat<BytesWritable,NullWritable> {
324 static class GeneratorInputSplit extends InputSplit implements Writable {
325 @Override
326 public long getLength() throws IOException, InterruptedException {
327 return 1;
328 }
329 @Override
330 public String[] getLocations() throws IOException, InterruptedException {
331 return new String[0];
332 }
333 @Override
334 public void readFields(DataInput arg0) throws IOException {
335 }
336 @Override
337 public void write(DataOutput arg0) throws IOException {
338 }
339 }
340
341 static class GeneratorRecordReader extends RecordReader<BytesWritable,NullWritable> {
342 private long count;
343 private long numNodes;
344 private Random64 rand;
345
346 @Override
347 public void close() throws IOException {
348 }
349
350 @Override
351 public BytesWritable getCurrentKey() throws IOException, InterruptedException {
352 byte[] bytes = new byte[ROWKEY_LENGTH];
353 rand.nextBytes(bytes);
354 return new BytesWritable(bytes);
355 }
356
357 @Override
358 public NullWritable getCurrentValue() throws IOException, InterruptedException {
359 return NullWritable.get();
360 }
361
362 @Override
363 public float getProgress() throws IOException, InterruptedException {
364 return (float)(count / (double)numNodes);
365 }
366
367 @Override
368 public void initialize(InputSplit arg0, TaskAttemptContext context)
369 throws IOException, InterruptedException {
370 numNodes = context.getConfiguration().getLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, 25000000);
371
372 rand = new Random64();
373 }
374
375 @Override
376 public boolean nextKeyValue() throws IOException, InterruptedException {
377 return count++ < numNodes;
378 }
379
380 }
381
382 @Override
383 public RecordReader<BytesWritable,NullWritable> createRecordReader(
384 InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
385 GeneratorRecordReader rr = new GeneratorRecordReader();
386 rr.initialize(split, context);
387 return rr;
388 }
389
390 @Override
391 public List<InputSplit> getSplits(JobContext job) throws IOException, InterruptedException {
392 int numMappers = job.getConfiguration().getInt(GENERATOR_NUM_MAPPERS_KEY, 1);
393
394 ArrayList<InputSplit> splits = new ArrayList<InputSplit>(numMappers);
395
396 for (int i = 0; i < numMappers; i++) {
397 splits.add(new GeneratorInputSplit());
398 }
399
400 return splits;
401 }
402 }
403
404
405 static class OneFilePerMapperSFIF<K, V> extends SequenceFileInputFormat<K, V> {
406 @Override
407 protected boolean isSplitable(JobContext context, Path filename) {
408 return false;
409 }
410 }
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439 static class GeneratorMapper
440 extends Mapper<BytesWritable, NullWritable, NullWritable, NullWritable> {
441
442 byte[][] first = null;
443 byte[][] prev = null;
444 byte[][] current = null;
445 byte[] id;
446 long count = 0;
447 int i;
448 BufferedMutator mutator;
449 Connection connection;
450 long numNodes;
451 long wrap;
452 int width;
453 boolean multipleUnevenColumnFamilies;
454 byte[] tinyValue = new byte[] { 't' };
455 byte[] bigValue = null;
456 Configuration conf;
457
458 volatile boolean walkersStop;
459 int numWalkers;
460 volatile List<Long> flushedLoops = new ArrayList<>();
461 List<Thread> walkers = new ArrayList<>();
462
463 @Override
464 protected void setup(Context context) throws IOException, InterruptedException {
465 id = Bytes.toBytes("Job: "+context.getJobID() + " Task: " + context.getTaskAttemptID());
466 Configuration conf = context.getConfiguration();
467 connection = ConnectionFactory.createConnection(conf);
468 instantiateHTable();
469 this.width = context.getConfiguration().getInt(GENERATOR_WIDTH_KEY, WIDTH_DEFAULT);
470 current = new byte[this.width][];
471 int wrapMultiplier = context.getConfiguration().getInt(GENERATOR_WRAP_KEY, WRAP_DEFAULT);
472 this.wrap = (long)wrapMultiplier * width;
473 this.numNodes = context.getConfiguration().getLong(
474 GENERATOR_NUM_ROWS_PER_MAP_KEY, (long)WIDTH_DEFAULT * WRAP_DEFAULT);
475 if (this.numNodes < this.wrap) {
476 this.wrap = this.numNodes;
477 }
478 this.multipleUnevenColumnFamilies = isMultiUnevenColumnFamilies(context.getConfiguration());
479 this.numWalkers = context.getConfiguration().getInt(CONCURRENT_WALKER_KEY, CONCURRENT_WALKER_DEFAULT);
480 this.walkersStop = false;
481 this.conf = context.getConfiguration();
482
483 if (multipleUnevenColumnFamilies) {
484 int n = context.getConfiguration().getInt(BIG_FAMILY_VALUE_SIZE_KEY, 256);
485 int limit = context.getConfiguration().getInt(
486 ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY,
487 ConnectionConfiguration.MAX_KEYVALUE_SIZE_DEFAULT);
488
489 Preconditions.checkArgument(
490 n <= limit,
491 "%s(%s) > %s(%s)",
492 BIG_FAMILY_VALUE_SIZE_KEY, n, ConnectionConfiguration.MAX_KEYVALUE_SIZE_KEY, limit);
493
494 bigValue = new byte[n];
495 ThreadLocalRandom.current().nextBytes(bigValue);
496 LOG.info("Create a bigValue with " + n + " bytes.");
497 }
498
499 Preconditions.checkArgument(
500 numNodes > 0,
501 "numNodes(%s) <= 0",
502 numNodes);
503 Preconditions.checkArgument(
504 numNodes % width == 0,
505 "numNodes(%s) mod width(%s) != 0",
506 numNodes, width);
507 Preconditions.checkArgument(
508 numNodes % wrap == 0,
509 "numNodes(%s) mod wrap(%s) != 0",
510 numNodes, wrap
511 );
512 }
513
514 protected void instantiateHTable() throws IOException {
515 mutator = connection.getBufferedMutator(
516 new BufferedMutatorParams(getTableName(connection.getConfiguration()))
517 .writeBufferSize(4 * 1024 * 1024));
518 }
519
520 @Override
521 protected void cleanup(Context context) throws IOException ,InterruptedException {
522 joinWalkers();
523 mutator.close();
524 connection.close();
525 }
526
527 @Override
528 protected void map(BytesWritable key, NullWritable value, Context output) throws IOException {
529 current[i] = new byte[key.getLength()];
530 System.arraycopy(key.getBytes(), 0, current[i], 0, key.getLength());
531 if (++i == current.length) {
532 LOG.debug("Persisting current.length=" + current.length + ", count=" + count +
533 ", id=" + Bytes.toStringBinary(id) + ", current=" + Bytes.toStringBinary(current[0]) +
534 ", i=" + i);
535 persist(output, count, prev, current, id);
536 i = 0;
537
538 if (first == null) {
539 first = current;
540 }
541 prev = current;
542 current = new byte[this.width][];
543
544 count += current.length;
545 output.setStatus("Count " + count);
546
547 if (count % wrap == 0) {
548
549
550 circularLeftShift(first);
551 persist(output, -1, prev, first, null);
552
553
554 if (numWalkers > 0) {
555 addFlushed(key.getBytes());
556 if (walkers.isEmpty()) {
557 startWalkers(numWalkers, conf, output);
558 }
559 }
560 first = null;
561 prev = null;
562 }
563 }
564 }
565
566 private static <T> void circularLeftShift(T[] first) {
567 T ez = first[0];
568 System.arraycopy(first, 1, first, 0, first.length - 1);
569 first[first.length - 1] = ez;
570 }
571
572 private void addFlushed(byte[] rowKey) {
573 synchronized (flushedLoops) {
574 flushedLoops.add(Bytes.toLong(rowKey));
575 flushedLoops.notifyAll();
576 }
577 }
578
579 protected void persist(Context output, long count, byte[][] prev, byte[][] current, byte[] id)
580 throws IOException {
581 for (int i = 0; i < current.length; i++) {
582
583 if (i % 100 == 0) {
584
585 output.progress();
586 }
587
588 Put put = new Put(current[i]);
589 put.addColumn(FAMILY_NAME, COLUMN_PREV, prev == null ? NO_KEY : prev[i]);
590
591 if (count >= 0) {
592 put.addColumn(FAMILY_NAME, COLUMN_COUNT, Bytes.toBytes(count + i));
593 }
594 if (id != null) {
595 put.addColumn(FAMILY_NAME, COLUMN_CLIENT, id);
596 }
597
598 if (this.multipleUnevenColumnFamilies) {
599
600 put.addColumn(TINY_FAMILY_NAME, TINY_FAMILY_NAME, this.tinyValue);
601
602 put.addColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME, this.bigValue);
603 }
604 mutator.mutate(put);
605 }
606
607 mutator.flush();
608 }
609
610 private void startWalkers(int numWalkers, Configuration conf, Context context) {
611 LOG.info("Starting " + numWalkers + " concurrent walkers");
612 for (int i = 0; i < numWalkers; i++) {
613 Thread walker = new Thread(new ContinuousConcurrentWalker(conf, context));
614 walker.start();
615 walkers.add(walker);
616 }
617 }
618
619 private void joinWalkers() {
620 walkersStop = true;
621 synchronized (flushedLoops) {
622 flushedLoops.notifyAll();
623 }
624 for (Thread walker : walkers) {
625 try {
626 walker.join();
627 } catch (InterruptedException e) {
628
629 }
630 }
631 }
632
633
634
635
636
637
638
639 public class ContinuousConcurrentWalker implements Runnable {
640
641 ConcurrentWalker walker;
642 Configuration conf;
643 Context context;
644 Random rand;
645
646 public ContinuousConcurrentWalker(Configuration conf, Context context) {
647 this.conf = conf;
648 this.context = context;
649 rand = new Random();
650 }
651
652 @Override
653 public void run() {
654 while (!walkersStop) {
655 try {
656 long node = selectLoop();
657 try {
658 walkLoop(node);
659 } catch (IOException e) {
660 context.getCounter(Counts.IOEXCEPTION).increment(1l);
661 return;
662 }
663 } catch (InterruptedException e) {
664 return;
665 }
666 }
667 }
668
669 private void walkLoop(long node) throws IOException {
670 walker = new ConcurrentWalker(context);
671 walker.setConf(conf);
672 walker.run(node, wrap);
673 }
674
675 private long selectLoop () throws InterruptedException{
676 synchronized (flushedLoops) {
677 while (flushedLoops.isEmpty() && !walkersStop) {
678 flushedLoops.wait();
679 }
680 if (walkersStop) {
681 throw new InterruptedException();
682 }
683 return flushedLoops.get(rand.nextInt(flushedLoops.size()));
684 }
685 }
686 }
687
688 public static class ConcurrentWalker extends WalkerBase {
689
690 Context context;
691
692 public ConcurrentWalker(Context context) {this.context = context;}
693
694 public void run(long startKeyIn, long maxQueriesIn) throws IOException {
695
696 long maxQueries = maxQueriesIn > 0 ? maxQueriesIn : Long.MAX_VALUE;
697 byte[] startKey = Bytes.toBytes(startKeyIn);
698
699 Connection connection = ConnectionFactory.createConnection(getConf());
700 Table table = connection.getTable(getTableName(getConf()));
701 long numQueries = 0;
702
703
704
705
706 CINode node = findStartNode(table, startKey);
707 if (node == null) {
708 LOG.error("Start node not found: " + Bytes.toStringBinary(startKey));
709 throw new IOException("Start node not found: " + startKeyIn);
710 }
711 while (numQueries < maxQueries) {
712 numQueries++;
713 byte[] prev = node.prev;
714 long t1 = System.currentTimeMillis();
715 node = getNode(prev, table, node);
716 long t2 = System.currentTimeMillis();
717 if (node == null) {
718 LOG.error("ConcurrentWalker found UNDEFINED NODE: " + Bytes.toStringBinary(prev));
719 context.getCounter(Counts.UNDEFINED).increment(1l);
720 } else if (node.prev.length == NO_KEY.length) {
721 LOG.error("ConcurrentWalker found TERMINATING NODE: " +
722 Bytes.toStringBinary(node.key));
723 context.getCounter(Counts.TERMINATING).increment(1l);
724 } else {
725
726 context.getCounter(Counts.SUCCESS).increment(1l);
727 }
728 }
729 table.close();
730 connection.close();
731 }
732 }
733 }
734
735 @Override
736 public int run(String[] args) throws Exception {
737 if (args.length < 3) {
738 System.err.println(USAGE);
739 return 1;
740 }
741 try {
742 int numMappers = Integer.parseInt(args[0]);
743 long numNodes = Long.parseLong(args[1]);
744 Path tmpOutput = new Path(args[2]);
745 Integer width = (args.length < 4) ? null : Integer.parseInt(args[3]);
746 Integer wrapMultiplier = (args.length < 5) ? null : Integer.parseInt(args[4]);
747 Integer numWalkers = (args.length < 6) ? null : Integer.parseInt(args[5]);
748 return run(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
749 } catch (NumberFormatException e) {
750 System.err.println("Parsing generator arguments failed: " + e.getMessage());
751 System.err.println(USAGE);
752 return 1;
753 }
754 }
755
756 protected void createSchema() throws IOException {
757 Configuration conf = getConf();
758 TableName tableName = getTableName(conf);
759 try (Connection conn = ConnectionFactory.createConnection(conf);
760 Admin admin = conn.getAdmin()) {
761 if (!admin.tableExists(tableName)) {
762 HTableDescriptor htd = new HTableDescriptor(getTableName(getConf()));
763 htd.addFamily(new HColumnDescriptor(FAMILY_NAME));
764
765 htd.addFamily(new HColumnDescriptor(BIG_FAMILY_NAME));
766 htd.addFamily(new HColumnDescriptor(TINY_FAMILY_NAME));
767
768
769 if (conf.getBoolean(HBaseTestingUtility.PRESPLIT_TEST_TABLE_KEY,
770 HBaseTestingUtility.PRESPLIT_TEST_TABLE)) {
771 int numberOfServers = admin.getClusterStatus().getServers().size();
772 if (numberOfServers == 0) {
773 throw new IllegalStateException("No live regionservers");
774 }
775 int regionsPerServer = conf.getInt(HBaseTestingUtility.REGIONS_PER_SERVER_KEY,
776 HBaseTestingUtility.DEFAULT_REGIONS_PER_SERVER);
777 int totalNumberOfRegions = numberOfServers * regionsPerServer;
778 LOG.info("Number of live regionservers: " + numberOfServers + ", " +
779 "pre-splitting table into " + totalNumberOfRegions + " regions " +
780 "(default regions per server: " + regionsPerServer + ")");
781
782
783 byte[][] splits = new RegionSplitter.UniformSplit().split(totalNumberOfRegions);
784
785 admin.createTable(htd, splits);
786 } else {
787
788
789
790 admin.createTable(htd);
791 }
792 }
793 } catch (MasterNotRunningException e) {
794 LOG.error("Master not running", e);
795 throw new IOException(e);
796 }
797 }
798
799 public int runRandomInputGenerator(int numMappers, long numNodes, Path tmpOutput,
800 Integer width, Integer wrapMultiplier, Integer numWalkers)
801 throws Exception {
802 LOG.info("Running RandomInputGenerator with numMappers=" + numMappers
803 + ", numNodes=" + numNodes);
804 Job job = Job.getInstance(getConf());
805
806 job.setJobName("Random Input Generator");
807 job.setNumReduceTasks(0);
808 job.setJarByClass(getClass());
809
810 job.setInputFormatClass(GeneratorInputFormat.class);
811 job.setOutputKeyClass(BytesWritable.class);
812 job.setOutputValueClass(NullWritable.class);
813
814 setJobConf(job, numMappers, numNodes, width, wrapMultiplier, numWalkers);
815
816 job.setMapperClass(Mapper.class);
817
818 FileOutputFormat.setOutputPath(job, tmpOutput);
819 job.setOutputFormatClass(SequenceFileOutputFormat.class);
820 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(), Random64.class);
821
822 boolean success = jobCompletion(job);
823
824 return success ? 0 : 1;
825 }
826
827 public int runGenerator(int numMappers, long numNodes, Path tmpOutput,
828 Integer width, Integer wrapMultiplier, Integer numWalkers)
829 throws Exception {
830 LOG.info("Running Generator with numMappers=" + numMappers +", numNodes=" + numNodes);
831 createSchema();
832 job = Job.getInstance(getConf());
833
834 job.setJobName("Link Generator");
835 job.setNumReduceTasks(0);
836 job.setJarByClass(getClass());
837
838 FileInputFormat.setInputPaths(job, tmpOutput);
839 job.setInputFormatClass(OneFilePerMapperSFIF.class);
840 job.setOutputKeyClass(NullWritable.class);
841 job.setOutputValueClass(NullWritable.class);
842
843 setJobConf(job, numMappers, numNodes, width, wrapMultiplier, numWalkers);
844
845 setMapperForGenerator(job);
846
847 job.setOutputFormatClass(NullOutputFormat.class);
848
849 job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
850 TableMapReduceUtil.addDependencyJars(job);
851 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
852 AbstractHBaseTool.class);
853 TableMapReduceUtil.initCredentials(job);
854
855 boolean success = jobCompletion(job);
856
857 return success ? 0 : 1;
858 }
859
860 protected boolean jobCompletion(Job job) throws IOException, InterruptedException,
861 ClassNotFoundException {
862 boolean success = job.waitForCompletion(true);
863 return success;
864 }
865
866 protected void setMapperForGenerator(Job job) {
867 job.setMapperClass(GeneratorMapper.class);
868 }
869
870 public int run(int numMappers, long numNodes, Path tmpOutput,
871 Integer width, Integer wrapMultiplier, Integer numWalkers)
872 throws Exception {
873 int ret = runRandomInputGenerator(numMappers, numNodes, tmpOutput, width, wrapMultiplier,
874 numWalkers);
875 if (ret > 0) {
876 return ret;
877 }
878 return runGenerator(numMappers, numNodes, tmpOutput, width, wrapMultiplier, numWalkers);
879 }
880
881 public boolean verify() {
882 try {
883 Counters counters = job.getCounters();
884 if (counters == null) {
885 LOG.info("Counters object was null, Generator verification cannot be performed."
886 + " This is commonly a result of insufficient YARN configuration.");
887 return false;
888 }
889
890 if (counters.findCounter(Counts.TERMINATING).getValue() > 0 ||
891 counters.findCounter(Counts.UNDEFINED).getValue() > 0 ||
892 counters.findCounter(Counts.IOEXCEPTION).getValue() > 0) {
893 LOG.error("Concurrent walker failed to verify during Generation phase");
894 LOG.error("TERMINATING nodes: " + counters.findCounter(Counts.TERMINATING).getValue());
895 LOG.error("UNDEFINED nodes: " + counters.findCounter(Counts.UNDEFINED).getValue());
896 LOG.error("IOEXCEPTION nodes: " + counters.findCounter(Counts.IOEXCEPTION).getValue());
897 return false;
898 }
899 } catch (IOException e) {
900 LOG.info("Generator verification could not find counter");
901 return false;
902 }
903 return true;
904 }
905 }
906
907
908
909
910
911
912
913 static class Search extends Configured implements Tool {
914 private static final Log LOG = LogFactory.getLog(Search.class);
915 protected Job job;
916
917 private static void printUsage(final String error) {
918 if (error != null && error.length() > 0) System.out.println("ERROR: " + error);
919 System.err.println("Usage: search <KEYS_DIR> [<MAPPERS_COUNT>]");
920 }
921
922 @Override
923 public int run(String[] args) throws Exception {
924 if (args.length < 1 || args.length > 2) {
925 printUsage(null);
926 return 1;
927 }
928 Path inputDir = new Path(args[0]);
929 int numMappers = 1;
930 if (args.length > 1) {
931 numMappers = Integer.parseInt(args[1]);
932 }
933 return run(inputDir, numMappers);
934 }
935
936
937
938
939 public static class WALSearcher extends WALPlayer {
940 public WALSearcher(Configuration conf) {
941 super(conf);
942 }
943
944
945
946
947 public static class WALMapperSearcher extends WALMapper {
948 private SortedSet<byte []> keysToFind;
949 private AtomicInteger rows = new AtomicInteger(0);
950
951 @Override
952 public void setup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation>.Context context)
953 throws IOException {
954 super.setup(context);
955 try {
956 this.keysToFind = readKeysToSearch(context.getConfiguration());
957 LOG.info("Loaded keys to find: count=" + this.keysToFind.size());
958 } catch (InterruptedException e) {
959 throw new InterruptedIOException(e.toString());
960 }
961 }
962
963 @Override
964 protected boolean filter(Context context, Cell cell) {
965
966 byte [] row = new byte [cell.getRowLength()];
967 System.arraycopy(cell.getRowArray(), cell.getRowOffset(), row, 0, cell.getRowLength());
968 boolean b = this.keysToFind.contains(row);
969 if (b) {
970 String keyStr = Bytes.toStringBinary(row);
971 try {
972 LOG.info("Found cell=" + cell + " , walKey=" + context.getCurrentKey());
973 } catch (IOException|InterruptedException e) {
974 LOG.warn(e);
975 }
976 if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
977 context.getCounter(FOUND_GROUP_KEY, keyStr).increment(1);
978 }
979 context.getCounter(FOUND_GROUP_KEY, "CELL_WITH_MISSING_ROW").increment(1);
980 }
981 return b;
982 }
983 }
984
985
986 @Override
987 public Job createSubmittableJob(String[] args) throws IOException {
988 Job job = super.createSubmittableJob(args);
989
990 job.setJarByClass(WALMapperSearcher.class);
991 job.setMapperClass(WALMapperSearcher.class);
992 job.setOutputFormatClass(NullOutputFormat.class);
993 return job;
994 }
995 }
996
997 static final String FOUND_GROUP_KEY = "Found";
998 static final String SEARCHER_INPUTDIR_KEY = "searcher.keys.inputdir";
999
1000 public int run(Path inputDir, int numMappers) throws Exception {
1001 getConf().set(SEARCHER_INPUTDIR_KEY, inputDir.toString());
1002 SortedSet<byte []> keys = readKeysToSearch(getConf());
1003 if (keys.isEmpty()) throw new RuntimeException("No keys to find");
1004 LOG.info("Count of keys to find: " + keys.size());
1005 for(byte [] key: keys) LOG.info("Key: " + Bytes.toStringBinary(key));
1006 Path hbaseDir = new Path(getConf().get(HConstants.HBASE_DIR));
1007
1008 Path walsDir = new Path(hbaseDir, HConstants.HREGION_LOGDIR_NAME);
1009 Path oldWalsDir = new Path(hbaseDir, HConstants.HREGION_OLDLOGDIR_NAME);
1010 LOG.info("Running Search with keys inputDir=" + inputDir +", numMappers=" + numMappers +
1011 " against " + getConf().get(HConstants.HBASE_DIR));
1012 int ret = ToolRunner.run(new WALSearcher(getConf()), new String [] {walsDir.toString(), ""});
1013 if (ret != 0) return ret;
1014 return ToolRunner.run(new WALSearcher(getConf()), new String [] {oldWalsDir.toString(), ""});
1015 }
1016
1017 static SortedSet<byte []> readKeysToSearch(final Configuration conf)
1018 throws IOException, InterruptedException {
1019 Path keysInputDir = new Path(conf.get(SEARCHER_INPUTDIR_KEY));
1020 FileSystem fs = FileSystem.get(conf);
1021 SortedSet<byte []> result = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
1022 if (!fs.exists(keysInputDir)) {
1023 throw new FileNotFoundException(keysInputDir.toString());
1024 }
1025 if (!fs.isDirectory(keysInputDir)) {
1026 throw new UnsupportedOperationException("TODO");
1027 } else {
1028 RemoteIterator<LocatedFileStatus> iterator = fs.listFiles(keysInputDir, false);
1029 while(iterator.hasNext()) {
1030 LocatedFileStatus keyFileStatus = iterator.next();
1031
1032 if (keyFileStatus.getPath().getName().startsWith("_")) continue;
1033 result.addAll(readFileToSearch(conf, fs, keyFileStatus));
1034 }
1035 }
1036 return result;
1037 }
1038
1039 private static SortedSet<byte[]> readFileToSearch(final Configuration conf,
1040 final FileSystem fs, final LocatedFileStatus keyFileStatus) throws IOException,
1041 InterruptedException {
1042 SortedSet<byte []> result = new TreeSet<byte []>(Bytes.BYTES_COMPARATOR);
1043
1044
1045 TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
1046 try (SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader rr =
1047 new SequenceFileAsBinaryInputFormat.SequenceFileAsBinaryRecordReader()) {
1048 InputSplit is =
1049 new FileSplit(keyFileStatus.getPath(), 0, keyFileStatus.getLen(), new String [] {});
1050 rr.initialize(is, context);
1051 while (rr.nextKeyValue()) {
1052 rr.getCurrentKey();
1053 BytesWritable bw = rr.getCurrentValue();
1054 if (Verify.VerifyReducer.whichType(bw.getBytes()) == Verify.Counts.UNDEFINED) {
1055 byte[] key = new byte[rr.getCurrentKey().getLength()];
1056 System.arraycopy(rr.getCurrentKey().getBytes(), 0, key, 0, rr.getCurrentKey()
1057 .getLength());
1058 result.add(key);
1059 }
1060 }
1061 }
1062 return result;
1063 }
1064 }
1065
1066
1067
1068
1069
1070 static class Verify extends Configured implements Tool {
1071
1072 private static final Log LOG = LogFactory.getLog(Verify.class);
1073 protected static final BytesWritable DEF = new BytesWritable(new byte[] { 0 });
1074 protected static final BytesWritable DEF_LOST_FAMILIES = new BytesWritable(new byte[] { 1 });
1075
1076 protected Job job;
1077
1078 public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
1079 private BytesWritable row = new BytesWritable();
1080 private BytesWritable ref = new BytesWritable();
1081
1082 private boolean multipleUnevenColumnFamilies;
1083
1084 @Override
1085 protected void setup(
1086 Mapper<ImmutableBytesWritable, Result, BytesWritable, BytesWritable>.Context context)
1087 throws IOException, InterruptedException {
1088 this.multipleUnevenColumnFamilies = isMultiUnevenColumnFamilies(context.getConfiguration());
1089 }
1090
1091 @Override
1092 protected void map(ImmutableBytesWritable key, Result value, Context context)
1093 throws IOException ,InterruptedException {
1094 byte[] rowKey = key.get();
1095 row.set(rowKey, 0, rowKey.length);
1096 if (multipleUnevenColumnFamilies
1097 && (!value.containsColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME) || !value.containsColumn(
1098 TINY_FAMILY_NAME, TINY_FAMILY_NAME))) {
1099 context.write(row, DEF_LOST_FAMILIES);
1100 } else {
1101 context.write(row, DEF);
1102 }
1103 byte[] prev = value.getValue(FAMILY_NAME, COLUMN_PREV);
1104 if (prev != null && prev.length > 0) {
1105 ref.set(prev, 0, prev.length);
1106 context.write(ref, row);
1107 } else {
1108 LOG.warn(String.format("Prev is not set for: %s", Bytes.toStringBinary(rowKey)));
1109 }
1110 }
1111 }
1112
1113
1114
1115
1116
1117 public static enum Counts {
1118 UNREFERENCED, UNDEFINED, REFERENCED, CORRUPT, EXTRAREFERENCES, EXTRA_UNDEF_REFERENCES,
1119 LOST_FAMILIES
1120 }
1121
1122
1123
1124
1125
1126
1127 public static class VerifyReducer extends
1128 Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable> {
1129 private ArrayList<byte[]> refs = new ArrayList<byte[]>();
1130 private final BytesWritable UNREF = new BytesWritable(addPrefixFlag(
1131 Counts.UNREFERENCED.ordinal(), new byte[] {}));
1132 private final BytesWritable LOSTFAM = new BytesWritable(addPrefixFlag(
1133 Counts.LOST_FAMILIES.ordinal(), new byte[] {}));
1134
1135 private AtomicInteger rows = new AtomicInteger(0);
1136 private Connection connection;
1137
1138 @Override
1139 protected void setup(Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
1140 throws IOException, InterruptedException {
1141 super.setup(context);
1142 this.connection = ConnectionFactory.createConnection(context.getConfiguration());
1143 }
1144
1145 @Override
1146 protected void cleanup(
1147 Reducer<BytesWritable, BytesWritable, BytesWritable, BytesWritable>.Context context)
1148 throws IOException, InterruptedException {
1149 if (this.connection != null) {
1150 this.connection.close();
1151 }
1152 super.cleanup(context);
1153 }
1154
1155
1156
1157
1158
1159
1160
1161 public static byte[] addPrefixFlag(final int ordinal, final byte [] r) {
1162 byte[] prefix = Bytes.toBytes((short)ordinal);
1163 if (prefix.length != Bytes.SIZEOF_SHORT) {
1164 throw new RuntimeException("Unexpected size: " + prefix.length);
1165 }
1166 byte[] result = new byte[prefix.length + r.length];
1167 System.arraycopy(prefix, 0, result, 0, prefix.length);
1168 System.arraycopy(r, 0, result, prefix.length, r.length);
1169 return result;
1170 }
1171
1172
1173
1174
1175
1176
1177 public static Counts whichType(final byte [] bs) {
1178 int ordinal = Bytes.toShort(bs, 0, Bytes.SIZEOF_SHORT);
1179 return Counts.values()[ordinal];
1180 }
1181
1182
1183
1184
1185
1186 public static byte[] getRowOnly(BytesWritable bw) {
1187 byte[] bytes = new byte [bw.getLength() - Bytes.SIZEOF_SHORT];
1188 System.arraycopy(bw.getBytes(), Bytes.SIZEOF_SHORT, bytes, 0, bytes.length);
1189 return bytes;
1190 }
1191
1192 @Override
1193 public void reduce(BytesWritable key, Iterable<BytesWritable> values, Context context)
1194 throws IOException, InterruptedException {
1195 int defCount = 0;
1196 boolean lostFamilies = false;
1197 refs.clear();
1198 for (BytesWritable type : values) {
1199 if (type.getLength() == DEF.getLength()) {
1200 defCount++;
1201 if (type.getBytes()[0] == 1) {
1202 lostFamilies = true;
1203 }
1204 } else {
1205 byte[] bytes = new byte[type.getLength()];
1206 System.arraycopy(type.getBytes(), 0, bytes, 0, type.getLength());
1207 refs.add(bytes);
1208 }
1209 }
1210
1211
1212 StringBuilder refsSb = null;
1213 if (defCount == 0 || refs.size() != 1) {
1214 String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
1215 refsSb = dumpExtraInfoOnRefs(key, context, refs);
1216 LOG.error("LinkedListError: key=" + keyString + ", reference(s)=" +
1217 (refsSb != null? refsSb.toString(): ""));
1218 }
1219 if (lostFamilies) {
1220 String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
1221 LOG.error("LinkedListError: key=" + keyString + ", lost big or tiny families");
1222 context.getCounter(Counts.LOST_FAMILIES).increment(1);
1223 context.write(key, LOSTFAM);
1224 }
1225
1226 if (defCount == 0 && refs.size() > 0) {
1227
1228
1229
1230 for (int i = 0; i < refs.size(); i++) {
1231 byte[] bs = refs.get(i);
1232 int ordinal;
1233 if (i <= 0) {
1234 ordinal = Counts.UNDEFINED.ordinal();
1235 context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
1236 context.getCounter(Counts.UNDEFINED).increment(1);
1237 } else {
1238 ordinal = Counts.EXTRA_UNDEF_REFERENCES.ordinal();
1239 context.write(key, new BytesWritable(addPrefixFlag(ordinal, bs)));
1240 }
1241 }
1242 if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
1243
1244
1245
1246
1247 String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
1248 context.getCounter("undef", keyString).increment(1);
1249 }
1250 } else if (defCount > 0 && refs.size() == 0) {
1251
1252 context.write(key, UNREF);
1253 context.getCounter(Counts.UNREFERENCED).increment(1);
1254 if (rows.addAndGet(1) < MISSING_ROWS_TO_LOG) {
1255 String keyString = Bytes.toStringBinary(key.getBytes(), 0, key.getLength());
1256 context.getCounter("unref", keyString).increment(1);
1257 }
1258 } else {
1259 if (refs.size() > 1) {
1260
1261 for (int i = 1; i < refs.size(); i++) {
1262 context.write(key,
1263 new BytesWritable(addPrefixFlag(Counts.EXTRAREFERENCES.ordinal(), refs.get(i))));
1264 }
1265 context.getCounter(Counts.EXTRAREFERENCES).increment(refs.size() - 1);
1266 }
1267
1268 context.getCounter(Counts.REFERENCED).increment(1);
1269 }
1270 }
1271
1272
1273
1274
1275
1276
1277 private StringBuilder dumpExtraInfoOnRefs(final BytesWritable key, final Context context,
1278 final List<byte []> refs)
1279 throws IOException {
1280 StringBuilder refsSb = null;
1281 if (refs.isEmpty()) return refsSb;
1282 refsSb = new StringBuilder();
1283 String comma = "";
1284
1285
1286
1287 TableName tn = getTableName(context.getConfiguration());
1288 try (Table t = this.connection.getTable(tn)) {
1289 for (byte [] ref : refs) {
1290 Result r = t.get(new Get(ref));
1291 List<Cell> cells = r.listCells();
1292 String ts = (cells != null && !cells.isEmpty())?
1293 new java.util.Date(cells.get(0).getTimestamp()).toString(): "";
1294 byte [] b = r.getValue(FAMILY_NAME, COLUMN_CLIENT);
1295 String jobStr = (b != null && b.length > 0)? Bytes.toString(b): "";
1296 b = r.getValue(FAMILY_NAME, COLUMN_COUNT);
1297 long count = (b != null && b.length > 0)? Bytes.toLong(b): -1;
1298 b = r.getValue(FAMILY_NAME, COLUMN_PREV);
1299 String refRegionLocation = "";
1300 String keyRegionLocation = "";
1301 if (b != null && b.length > 0) {
1302 try (RegionLocator rl = this.connection.getRegionLocator(tn)) {
1303 HRegionLocation hrl = rl.getRegionLocation(b);
1304 if (hrl != null) refRegionLocation = hrl.toString();
1305
1306 hrl = rl.getRegionLocation(key.getBytes());
1307 if (hrl != null) keyRegionLocation = hrl.toString();
1308 }
1309 }
1310 LOG.error("Extras on ref without a def, ref=" + Bytes.toStringBinary(ref) +
1311 ", refPrevEqualsKey=" +
1312 (Bytes.compareTo(key.getBytes(), 0, key.getLength(), b, 0, b.length) == 0) +
1313 ", key=" + Bytes.toStringBinary(key.getBytes(), 0, key.getLength()) +
1314 ", ref row date=" + ts + ", jobStr=" + jobStr +
1315 ", ref row count=" + count +
1316 ", ref row regionLocation=" + refRegionLocation +
1317 ", key row regionLocation=" + keyRegionLocation);
1318 refsSb.append(comma);
1319 comma = ",";
1320 refsSb.append(Bytes.toStringBinary(ref));
1321 }
1322 }
1323 return refsSb;
1324 }
1325 }
1326
1327 @Override
1328 public int run(String[] args) throws Exception {
1329 if (args.length != 2) {
1330 System.out.println("Usage : " + Verify.class.getSimpleName()
1331 + " <output dir> <num reducers>");
1332 return 0;
1333 }
1334
1335 String outputDir = args[0];
1336 int numReducers = Integer.parseInt(args[1]);
1337
1338 return run(outputDir, numReducers);
1339 }
1340
1341 public int run(String outputDir, int numReducers) throws Exception {
1342 return run(new Path(outputDir), numReducers);
1343 }
1344
1345 public int run(Path outputDir, int numReducers) throws Exception {
1346 LOG.info("Running Verify with outputDir=" + outputDir +", numReducers=" + numReducers);
1347
1348 job = Job.getInstance(getConf());
1349
1350 job.setJobName("Link Verifier");
1351 job.setNumReduceTasks(numReducers);
1352 job.setJarByClass(getClass());
1353
1354 setJobScannerConf(job);
1355
1356 Scan scan = new Scan();
1357 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
1358 scan.setCaching(10000);
1359 scan.setCacheBlocks(false);
1360 if (isMultiUnevenColumnFamilies(getConf())) {
1361 scan.addColumn(BIG_FAMILY_NAME, BIG_FAMILY_NAME);
1362 scan.addColumn(TINY_FAMILY_NAME, TINY_FAMILY_NAME);
1363 }
1364
1365 TableMapReduceUtil.initTableMapperJob(getTableName(getConf()).getName(), scan,
1366 VerifyMapper.class, BytesWritable.class, BytesWritable.class, job);
1367 TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
1368 AbstractHBaseTool.class);
1369
1370 job.getConfiguration().setBoolean("mapreduce.map.speculative", false);
1371
1372 job.setReducerClass(VerifyReducer.class);
1373 job.setOutputFormatClass(SequenceFileAsBinaryOutputFormat.class);
1374 job.setOutputKeyClass(BytesWritable.class);
1375 job.setOutputValueClass(BytesWritable.class);
1376 TextOutputFormat.setOutputPath(job, outputDir);
1377
1378 boolean success = job.waitForCompletion(true);
1379
1380 if (success) {
1381 Counters counters = job.getCounters();
1382 if (null == counters) {
1383 LOG.warn("Counters were null, cannot verify Job completion."
1384 + " This is commonly a result of insufficient YARN configuration.");
1385
1386 return 0;
1387 }
1388
1389
1390 if (verifyUnexpectedValues(counters)) {
1391
1392 return 0;
1393 }
1394 }
1395
1396
1397 return 1;
1398 }
1399
1400 public boolean verify(long expectedReferenced) throws Exception {
1401 if (job == null) {
1402 throw new IllegalStateException("You should call run() first");
1403 }
1404
1405 Counters counters = job.getCounters();
1406 if (counters == null) {
1407 LOG.info("Counters object was null, write verification cannot be performed."
1408 + " This is commonly a result of insufficient YARN configuration.");
1409 return false;
1410 }
1411
1412
1413 boolean success = verifyExpectedValues(expectedReferenced, counters);
1414
1415 if (!verifyUnexpectedValues(counters)) {
1416
1417 success = false;
1418 }
1419
1420 if (!success) {
1421 handleFailure(counters);
1422 }
1423 return success;
1424 }
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435 protected boolean verifyExpectedValues(long expectedReferenced, Counters counters) {
1436 final Counter referenced = counters.findCounter(Counts.REFERENCED);
1437 final Counter unreferenced = counters.findCounter(Counts.UNREFERENCED);
1438 boolean success = true;
1439
1440 if (expectedReferenced != referenced.getValue()) {
1441 LOG.error("Expected referenced count does not match with actual referenced count. " +
1442 "expected referenced=" + expectedReferenced + " ,actual=" + referenced.getValue());
1443 success = false;
1444 }
1445
1446 if (unreferenced.getValue() > 0) {
1447 final Counter multiref = counters.findCounter(Counts.EXTRAREFERENCES);
1448 boolean couldBeMultiRef = (multiref.getValue() == unreferenced.getValue());
1449 LOG.error("Unreferenced nodes were not expected. Unreferenced count=" + unreferenced.getValue()
1450 + (couldBeMultiRef ? "; could be due to duplicate random numbers" : ""));
1451 success = false;
1452 }
1453
1454 return success;
1455 }
1456
1457
1458
1459
1460
1461
1462
1463
1464 protected boolean verifyUnexpectedValues(Counters counters) {
1465 final Counter undefined = counters.findCounter(Counts.UNDEFINED);
1466 final Counter lostfamilies = counters.findCounter(Counts.LOST_FAMILIES);
1467 boolean success = true;
1468
1469 if (undefined.getValue() > 0) {
1470 LOG.error("Found an undefined node. Undefined count=" + undefined.getValue());
1471 success = false;
1472 }
1473
1474 if (lostfamilies.getValue() > 0) {
1475 LOG.error("Found nodes which lost big or tiny families, count=" + lostfamilies.getValue());
1476 success = false;
1477 }
1478
1479 return success;
1480 }
1481
1482 protected void handleFailure(Counters counters) throws IOException {
1483 Configuration conf = job.getConfiguration();
1484 TableName tableName = getTableName(conf);
1485 try (Connection conn = ConnectionFactory.createConnection(conf)) {
1486 try (RegionLocator rl = conn.getRegionLocator(tableName)) {
1487 CounterGroup g = counters.getGroup("undef");
1488 Iterator<Counter> it = g.iterator();
1489 while (it.hasNext()) {
1490 String keyString = it.next().getName();
1491 byte[] key = Bytes.toBytes(keyString);
1492 HRegionLocation loc = rl.getRegionLocation(key, true);
1493 LOG.error("undefined row " + keyString + ", " + loc);
1494 }
1495 g = counters.getGroup("unref");
1496 it = g.iterator();
1497 while (it.hasNext()) {
1498 String keyString = it.next().getName();
1499 byte[] key = Bytes.toBytes(keyString);
1500 HRegionLocation loc = rl.getRegionLocation(key, true);
1501 LOG.error("unreferred row " + keyString + ", " + loc);
1502 }
1503 }
1504 }
1505 }
1506 }
1507
1508
1509
1510
1511
1512 static class Loop extends Configured implements Tool {
1513
1514 private static final Log LOG = LogFactory.getLog(Loop.class);
1515 private static final String USAGE = "Usage: Loop <num iterations> <num mappers> " +
1516 "<num nodes per mapper> <output dir> <num reducers> [<width> <wrap multiplier>" +
1517 " <num walker threads>] \n" +
1518 "where <num nodes per map> should be a multiple of width*wrap multiplier, 25M by default \n" +
1519 "walkers will select and verify random flushed loop during Generation.";
1520
1521 IntegrationTestBigLinkedList it;
1522
1523 protected void runGenerator(int numMappers, long numNodes,
1524 String outputDir, Integer width, Integer wrapMultiplier, Integer numWalkers)
1525 throws Exception {
1526 Path outputPath = new Path(outputDir);
1527 UUID uuid = UUID.randomUUID();
1528 Path generatorOutput = new Path(outputPath, uuid.toString());
1529
1530 Generator generator = new Generator();
1531 generator.setConf(getConf());
1532 int retCode = generator.run(numMappers, numNodes, generatorOutput, width, wrapMultiplier,
1533 numWalkers);
1534 if (retCode > 0) {
1535 throw new RuntimeException("Generator failed with return code: " + retCode);
1536 }
1537 if (numWalkers > 0) {
1538 if (!generator.verify()) {
1539 throw new RuntimeException("Generator.verify failed");
1540 }
1541 }
1542 }
1543
1544 protected void runVerify(String outputDir,
1545 int numReducers, long expectedNumNodes) throws Exception {
1546 Path outputPath = new Path(outputDir);
1547 UUID uuid = UUID.randomUUID();
1548 Path iterationOutput = new Path(outputPath, uuid.toString());
1549
1550 Verify verify = new Verify();
1551 verify.setConf(getConf());
1552 int retCode = verify.run(iterationOutput, numReducers);
1553 if (retCode > 0) {
1554 throw new RuntimeException("Verify.run failed with return code: " + retCode);
1555 }
1556
1557 if (!verify.verify(expectedNumNodes)) {
1558 throw new RuntimeException("Verify.verify failed");
1559 }
1560 LOG.info("Verify finished with success. Total nodes=" + expectedNumNodes);
1561 }
1562
1563 @Override
1564 public int run(String[] args) throws Exception {
1565 if (args.length < 5) {
1566 System.err.println(USAGE);
1567 return 1;
1568 }
1569 try {
1570 int numIterations = Integer.parseInt(args[0]);
1571 int numMappers = Integer.parseInt(args[1]);
1572 long numNodes = Long.parseLong(args[2]);
1573 String outputDir = args[3];
1574 int numReducers = Integer.parseInt(args[4]);
1575 Integer width = (args.length < 6) ? null : Integer.parseInt(args[5]);
1576 Integer wrapMultiplier = (args.length < 7) ? null : Integer.parseInt(args[6]);
1577 Integer numWalkers = (args.length < 8) ? 0 : Integer.parseInt(args[7]);
1578
1579 long expectedNumNodes = 0;
1580
1581 if (numIterations < 0) {
1582 numIterations = Integer.MAX_VALUE;
1583 }
1584 LOG.info("Running Loop with args:" + Arrays.deepToString(args));
1585 for (int i = 0; i < numIterations; i++) {
1586 LOG.info("Starting iteration = " + i);
1587 runGenerator(numMappers, numNodes, outputDir, width, wrapMultiplier, numWalkers);
1588 expectedNumNodes += numMappers * numNodes;
1589 runVerify(outputDir, numReducers, expectedNumNodes);
1590 }
1591 return 0;
1592 } catch (NumberFormatException e) {
1593 System.err.println("Parsing loop arguments failed: " + e.getMessage());
1594 System.err.println(USAGE);
1595 return 1;
1596 }
1597 }
1598 }
1599
1600
1601
1602
1603 private static class Print extends Configured implements Tool {
1604 @Override
1605 public int run(String[] args) throws Exception {
1606 Options options = new Options();
1607 options.addOption("s", "start", true, "start key");
1608 options.addOption("e", "end", true, "end key");
1609 options.addOption("l", "limit", true, "number to print");
1610
1611 GnuParser parser = new GnuParser();
1612 CommandLine cmd = null;
1613 try {
1614 cmd = parser.parse(options, args);
1615 if (cmd.getArgs().length != 0) {
1616 throw new ParseException("Command takes no arguments");
1617 }
1618 } catch (ParseException e) {
1619 System.err.println("Failed to parse command line " + e.getMessage());
1620 System.err.println();
1621 HelpFormatter formatter = new HelpFormatter();
1622 formatter.printHelp(getClass().getSimpleName(), options);
1623 System.exit(-1);
1624 }
1625
1626 Table table = new HTable(getConf(), getTableName(getConf()));
1627
1628 Scan scan = new Scan();
1629 scan.setBatch(10000);
1630
1631 if (cmd.hasOption("s"))
1632 scan.setStartRow(Bytes.toBytesBinary(cmd.getOptionValue("s")));
1633
1634 if (cmd.hasOption("e"))
1635 scan.setStopRow(Bytes.toBytesBinary(cmd.getOptionValue("e")));
1636
1637 int limit = 0;
1638 if (cmd.hasOption("l"))
1639 limit = Integer.parseInt(cmd.getOptionValue("l"));
1640 else
1641 limit = 100;
1642
1643 ResultScanner scanner = table.getScanner(scan);
1644
1645 CINode node = new CINode();
1646 Result result = scanner.next();
1647 int count = 0;
1648 while (result != null && count++ < limit) {
1649 node = getCINode(result, node);
1650 System.out.printf("%s:%s:%012d:%s\n", Bytes.toStringBinary(node.key),
1651 Bytes.toStringBinary(node.prev), node.count, node.client);
1652 result = scanner.next();
1653 }
1654 scanner.close();
1655 table.close();
1656
1657 return 0;
1658 }
1659 }
1660
1661
1662
1663
1664 private static class Delete extends Configured implements Tool {
1665 @Override
1666 public int run(String[] args) throws Exception {
1667 if (args.length != 1) {
1668 System.out.println("Usage : " + Delete.class.getSimpleName() + " <node to delete>");
1669 return 0;
1670 }
1671 byte[] val = Bytes.toBytesBinary(args[0]);
1672
1673 org.apache.hadoop.hbase.client.Delete delete
1674 = new org.apache.hadoop.hbase.client.Delete(val);
1675
1676 Table table = new HTable(getConf(), getTableName(getConf()));
1677 table.delete(delete);
1678 table.close();
1679
1680 System.out.println("Delete successful");
1681 return 0;
1682 }
1683 }
1684
1685 abstract static class WalkerBase extends Configured{
1686 protected static CINode findStartNode(Table table, byte[] startKey) throws IOException {
1687 Scan scan = new Scan();
1688 scan.setStartRow(startKey);
1689 scan.setBatch(1);
1690 scan.addColumn(FAMILY_NAME, COLUMN_PREV);
1691
1692 long t1 = System.currentTimeMillis();
1693 ResultScanner scanner = table.getScanner(scan);
1694 Result result = scanner.next();
1695 long t2 = System.currentTimeMillis();
1696 scanner.close();
1697
1698 if ( result != null) {
1699 CINode node = getCINode(result, new CINode());
1700 System.out.printf("FSR %d %s\n", t2 - t1, Bytes.toStringBinary(node.key));
1701 return node;
1702 }
1703
1704 System.out.println("FSR " + (t2 - t1));
1705
1706 return null;
1707 }
1708 protected CINode getNode(byte[] row, Table table, CINode node) throws IOException {
1709 Get get = new Get(row);
1710 get.addColumn(FAMILY_NAME, COLUMN_PREV);
1711 Result result = table.get(get);
1712 return getCINode(result, node);
1713 }
1714 }
1715
1716
1717
1718
1719 private static class Walker extends WalkerBase implements Tool {
1720
1721 public Walker(){}
1722
1723 @Override
1724 public int run(String[] args) throws IOException {
1725
1726 Options options = new Options();
1727 options.addOption("n", "num", true, "number of queries");
1728 options.addOption("s", "start", true, "key to start at, binary string");
1729 options.addOption("l", "logevery", true, "log every N queries");
1730
1731 GnuParser parser = new GnuParser();
1732 CommandLine cmd = null;
1733 try {
1734 cmd = parser.parse(options, args);
1735 if (cmd.getArgs().length != 0) {
1736 throw new ParseException("Command takes no arguments");
1737 }
1738 } catch (ParseException e) {
1739 System.err.println("Failed to parse command line " + e.getMessage());
1740 System.err.println();
1741 HelpFormatter formatter = new HelpFormatter();
1742 formatter.printHelp(getClass().getSimpleName(), options);
1743 System.exit(-1);
1744 }
1745
1746 long maxQueries = Long.MAX_VALUE;
1747 if (cmd.hasOption('n')) {
1748 maxQueries = Long.parseLong(cmd.getOptionValue("n"));
1749 }
1750 Random rand = new SecureRandom();
1751 boolean isSpecificStart = cmd.hasOption('s');
1752
1753 byte[] startKey = isSpecificStart ? Bytes.toBytesBinary(cmd.getOptionValue('s')) : null;
1754 int logEvery = cmd.hasOption('l') ? Integer.parseInt(cmd.getOptionValue('l')) : 1;
1755
1756 Table table = new HTable(getConf(), getTableName(getConf()));
1757 long numQueries = 0;
1758
1759
1760
1761 while (numQueries < maxQueries && (numQueries == 0 || !isSpecificStart)) {
1762 if (!isSpecificStart) {
1763 startKey = new byte[ROWKEY_LENGTH];
1764 rand.nextBytes(startKey);
1765 }
1766 CINode node = findStartNode(table, startKey);
1767 if (node == null && isSpecificStart) {
1768 System.err.printf("Start node not found: %s \n", Bytes.toStringBinary(startKey));
1769 }
1770 numQueries++;
1771 while (node != null && node.prev.length != NO_KEY.length &&
1772 numQueries < maxQueries) {
1773 byte[] prev = node.prev;
1774 long t1 = System.currentTimeMillis();
1775 node = getNode(prev, table, node);
1776 long t2 = System.currentTimeMillis();
1777 if (logEvery > 0 && numQueries % logEvery == 0) {
1778 System.out.printf("CQ %d: %d %s \n", numQueries, t2 - t1, Bytes.toStringBinary(prev));
1779 }
1780 numQueries++;
1781 if (node == null) {
1782 System.err.printf("UNDEFINED NODE %s \n", Bytes.toStringBinary(prev));
1783 } else if (node.prev.length == NO_KEY.length) {
1784 System.err.printf("TERMINATING NODE %s \n", Bytes.toStringBinary(node.key));
1785 }
1786 }
1787 }
1788 table.close();
1789 return 0;
1790 }
1791 }
1792
1793 private static class Clean extends Configured implements Tool {
1794 @Override public int run(String[] args) throws Exception {
1795 if (args.length < 1) {
1796 System.err.println("Usage: Clean <output dir>");
1797 return -1;
1798 }
1799
1800 Path p = new Path(args[0]);
1801 Configuration conf = getConf();
1802 TableName tableName = getTableName(conf);
1803 try (FileSystem fs = HFileSystem.get(conf);
1804 Connection conn = ConnectionFactory.createConnection(conf);
1805 Admin admin = conn.getAdmin()) {
1806 if (admin.tableExists(tableName)) {
1807 admin.disableTable(tableName);
1808 admin.deleteTable(tableName);
1809 }
1810
1811 if (fs.exists(p)) {
1812 fs.delete(p, true);
1813 }
1814 }
1815
1816 return 0;
1817 }
1818 }
1819
1820 static TableName getTableName(Configuration conf) {
1821 return TableName.valueOf(conf.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1822 }
1823
1824 private static CINode getCINode(Result result, CINode node) {
1825 node.key = Bytes.copy(result.getRow());
1826 if (result.containsColumn(FAMILY_NAME, COLUMN_PREV)) {
1827 node.prev = Bytes.copy(result.getValue(FAMILY_NAME, COLUMN_PREV));
1828 } else {
1829 node.prev = NO_KEY;
1830 }
1831 if (result.containsColumn(FAMILY_NAME, COLUMN_COUNT)) {
1832 node.count = Bytes.toLong(result.getValue(FAMILY_NAME, COLUMN_COUNT));
1833 } else {
1834 node.count = -1;
1835 }
1836 if (result.containsColumn(FAMILY_NAME, COLUMN_CLIENT)) {
1837 node.client = Bytes.toString(result.getValue(FAMILY_NAME, COLUMN_CLIENT));
1838 } else {
1839 node.client = "";
1840 }
1841 return node;
1842 }
1843
1844 protected IntegrationTestingUtility util;
1845
1846 @Override
1847 public void setUpCluster() throws Exception {
1848 util = getTestingUtil(getConf());
1849 boolean isDistributed = util.isDistributedCluster();
1850 util.initializeCluster(isDistributed ? 1 : this.NUM_SLAVES_BASE);
1851 if (!isDistributed) {
1852 util.startMiniMapReduceCluster();
1853 }
1854 this.setConf(util.getConfiguration());
1855 }
1856
1857 @Override
1858 public void cleanUpCluster() throws Exception {
1859 super.cleanUpCluster();
1860 if (util.isDistributedCluster()) {
1861 util.shutdownMiniMapReduceCluster();
1862 }
1863 }
1864
1865 private static boolean isMultiUnevenColumnFamilies(Configuration conf) {
1866 return conf.getBoolean(Generator.MULTIPLE_UNEVEN_COLUMNFAMILIES_KEY,true);
1867 }
1868
1869 @Test
1870 public void testContinuousIngest() throws IOException, Exception {
1871
1872 Configuration conf = getTestingUtil(getConf()).getConfiguration();
1873 if (isMultiUnevenColumnFamilies(getConf())) {
1874
1875 conf.set(FlushPolicyFactory.HBASE_FLUSH_POLICY_KEY, FlushLargeStoresPolicy.class.getName());
1876 }
1877 int ret =
1878 ToolRunner.run(conf, new Loop(), new String[] { "1", "1", "2000000",
1879 util.getDataTestDirOnTestFS("IntegrationTestBigLinkedList").toString(), "1" });
1880 org.junit.Assert.assertEquals(0, ret);
1881 }
1882
1883 private void usage() {
1884 System.err.println("Usage: " + this.getClass().getSimpleName() + " COMMAND [COMMAND options]");
1885 printCommands();
1886 }
1887
1888 private void printCommands() {
1889 System.err.println("Commands:");
1890 System.err.println(" generator Map only job that generates data.");
1891 System.err.println(" verify A map reduce job that looks for holes. Check return code and");
1892 System.err.println(" look at the counts after running. See REFERENCED and");
1893 System.err.println(" UNREFERENCED are ok. Any UNDEFINED counts are bad. Do not run");
1894 System.err.println(" with the Generator.");
1895 System.err.println(" walker " +
1896 "Standalone program that starts following a linked list & emits timing info.");
1897 System.err.println(" print Standalone program that prints nodes in the linked list.");
1898 System.err.println(" delete Standalone program that deletes a single node.");
1899 System.err.println(" loop Program to Loop through Generator and Verify steps");
1900 System.err.println(" clean Program to clean all left over detritus.");
1901 System.err.println(" search Search for missing keys.");
1902 System.err.flush();
1903 }
1904
1905 @Override
1906 protected void processOptions(CommandLine cmd) {
1907 super.processOptions(cmd);
1908 String[] args = cmd.getArgs();
1909
1910 if (args.length < 1) {
1911 printUsage(this.getClass().getSimpleName() +
1912 " <general options> COMMAND [<COMMAND options>]", "General options:", "");
1913 printCommands();
1914
1915 throw new RuntimeException("Incorrect Number of args.");
1916 }
1917 toRun = args[0];
1918 otherArgs = Arrays.copyOfRange(args, 1, args.length);
1919 }
1920
1921 @Override
1922 public int runTestFromCommandLine() throws Exception {
1923 Tool tool = null;
1924 if (toRun.equalsIgnoreCase("Generator")) {
1925 tool = new Generator();
1926 } else if (toRun.equalsIgnoreCase("Verify")) {
1927 tool = new Verify();
1928 } else if (toRun.equalsIgnoreCase("Loop")) {
1929 Loop loop = new Loop();
1930 loop.it = this;
1931 tool = loop;
1932 } else if (toRun.equalsIgnoreCase("Walker")) {
1933 tool = new Walker();
1934 } else if (toRun.equalsIgnoreCase("Print")) {
1935 tool = new Print();
1936 } else if (toRun.equalsIgnoreCase("Delete")) {
1937 tool = new Delete();
1938 } else if (toRun.equalsIgnoreCase("Clean")) {
1939 tool = new Clean();
1940 } else if (toRun.equalsIgnoreCase("Search")) {
1941 tool = new Search();
1942 } else {
1943 usage();
1944 throw new RuntimeException("Unknown arg");
1945 }
1946
1947 return ToolRunner.run(getConf(), tool, otherArgs);
1948 }
1949
1950 @Override
1951 public TableName getTablename() {
1952 Configuration c = getConf();
1953 return TableName.valueOf(c.get(TABLE_NAME_KEY, DEFAULT_TABLE_NAME));
1954 }
1955
1956 @Override
1957 protected Set<String> getColumnFamilies() {
1958 if (isMultiUnevenColumnFamilies(getConf())) {
1959 return Sets.newHashSet(Bytes.toString(FAMILY_NAME), Bytes.toString(BIG_FAMILY_NAME),
1960 Bytes.toString(TINY_FAMILY_NAME));
1961 } else {
1962 return Sets.newHashSet(Bytes.toString(FAMILY_NAME));
1963 }
1964 }
1965
1966 private static void setJobConf(Job job, int numMappers, long numNodes,
1967 Integer width, Integer wrapMultiplier, Integer numWalkers) {
1968 job.getConfiguration().setInt(GENERATOR_NUM_MAPPERS_KEY, numMappers);
1969 job.getConfiguration().setLong(GENERATOR_NUM_ROWS_PER_MAP_KEY, numNodes);
1970 if (width != null) {
1971 job.getConfiguration().setInt(GENERATOR_WIDTH_KEY, width);
1972 }
1973 if (wrapMultiplier != null) {
1974 job.getConfiguration().setInt(GENERATOR_WRAP_KEY, wrapMultiplier);
1975 }
1976 if (numWalkers != null) {
1977 job.getConfiguration().setInt(CONCURRENT_WALKER_KEY, numWalkers);
1978 }
1979 }
1980
1981 public static void setJobScannerConf(Job job) {
1982
1983 job.getConfiguration().setBoolean(ScannerCallable.LOG_SCANNER_ACTIVITY, true);
1984 job.getConfiguration().setInt(TableRecordReaderImpl.LOG_PER_ROW_COUNT, 100000);
1985 }
1986
1987 public static void main(String[] args) throws Exception {
1988 Configuration conf = HBaseConfiguration.create();
1989 IntegrationTestingUtility.setUseDistributedCluster(conf);
1990 int ret = ToolRunner.run(conf, new IntegrationTestBigLinkedList(), args);
1991 System.exit(ret);
1992 }
1993 }