1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17 package org.apache.hadoop.hbase.regionserver;
18
19 import java.io.IOException;
20 import java.util.ArrayList;
21 import java.util.Arrays;
22 import java.util.Collections;
23 import java.util.List;
24 import java.util.Random;
25 import java.util.SortedSet;
26 import java.util.concurrent.Callable;
27 import java.util.concurrent.ConcurrentSkipListSet;
28 import java.util.concurrent.ExecutionException;
29 import java.util.concurrent.ExecutorCompletionService;
30 import java.util.concurrent.ExecutorService;
31 import java.util.concurrent.Executors;
32 import java.util.concurrent.Future;
33 import java.util.concurrent.TimeUnit;
34 import java.util.concurrent.atomic.AtomicLong;
35
36 import org.apache.commons.cli.CommandLine;
37 import org.apache.commons.cli.CommandLineParser;
38 import org.apache.commons.cli.HelpFormatter;
39 import org.apache.commons.cli.Option;
40 import org.apache.commons.cli.OptionGroup;
41 import org.apache.commons.cli.Options;
42 import org.apache.commons.cli.ParseException;
43 import org.apache.commons.cli.PosixParser;
44 import org.apache.commons.logging.Log;
45 import org.apache.commons.logging.LogFactory;
46 import org.apache.hadoop.conf.Configuration;
47 import org.apache.hadoop.fs.FileSystem;
48 import org.apache.hadoop.fs.Path;
49 import org.apache.hadoop.hbase.Cell;
50 import org.apache.hadoop.hbase.HBaseConfiguration;
51 import org.apache.hadoop.hbase.HColumnDescriptor;
52 import org.apache.hadoop.hbase.HRegionInfo;
53 import org.apache.hadoop.hbase.HTableDescriptor;
54 import org.apache.hadoop.hbase.KeyValue;
55 import org.apache.hadoop.hbase.KeyValueUtil;
56 import org.apache.hadoop.hbase.TableName;
57 import org.apache.hadoop.hbase.client.Scan;
58 import org.apache.hadoop.hbase.io.compress.Compression;
59 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
60 import org.apache.hadoop.hbase.io.hfile.BlockCache;
61 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
62 import org.apache.hadoop.hbase.io.hfile.HFile;
63 import org.apache.hadoop.hbase.io.hfile.HFilePrettyPrinter;
64 import org.apache.hadoop.hbase.util.Bytes;
65 import org.apache.hadoop.hbase.util.LoadTestTool;
66 import org.apache.hadoop.hbase.util.MD5Hash;
67 import org.apache.hadoop.util.StringUtils;
68
69
70
71
72 public class HFileReadWriteTest {
73
74 private static final String TABLE_NAME = "MyTable";
75
76 private static enum Workload {
77 MERGE("merge", "Merge the specified HFiles", 1, Integer.MAX_VALUE),
78 RANDOM_READS("read", "Perform a random read benchmark on the given HFile",
79 1, 1);
80
81 private String option;
82 private String description;
83
84 public final int minNumInputFiles;
85 public final int maxNumInputFiles;
86
87 Workload(String option, String description, int minNumInputFiles,
88 int maxNumInputFiles) {
89 this.option = option;
90 this.description = description;
91 this.minNumInputFiles = minNumInputFiles;
92 this.maxNumInputFiles = maxNumInputFiles;
93 }
94
95 static OptionGroup getOptionGroup() {
96 OptionGroup optionGroup = new OptionGroup();
97 for (Workload w : values())
98 optionGroup.addOption(new Option(w.option, w.description));
99 return optionGroup;
100 }
101
102 private static String getOptionListStr() {
103 StringBuilder sb = new StringBuilder();
104 for (Workload w : values()) {
105 if (sb.length() > 0)
106 sb.append(", ");
107 sb.append("-" + w.option);
108 }
109 return sb.toString();
110 }
111
112 static Workload fromCmdLine(CommandLine cmdLine) {
113 for (Workload w : values()) {
114 if (cmdLine.hasOption(w.option))
115 return w;
116 }
117 LOG.error("No workload specified. Specify one of the options: " +
118 getOptionListStr());
119 return null;
120 }
121
122 public String onlyUsedFor() {
123 return ". Only used for the " + this + " workload.";
124 }
125 }
126
127 private static final String OUTPUT_DIR_OPTION = "output_dir";
128 private static final String COMPRESSION_OPTION = "compression";
129 private static final String BLOOM_FILTER_OPTION = "bloom";
130 private static final String BLOCK_SIZE_OPTION = "block_size";
131 private static final String DURATION_OPTION = "duration";
132 private static final String NUM_THREADS_OPTION = "num_threads";
133
134 private static final Log LOG = LogFactory.getLog(HFileReadWriteTest.class);
135
136 private Workload workload;
137 private FileSystem fs;
138 private Configuration conf;
139 private CacheConfig cacheConf;
140 private List<String> inputFileNames;
141 private Path outputDir;
142 private int numReadThreads;
143 private int durationSec;
144 private DataBlockEncoding dataBlockEncoding;
145
146 private BloomType bloomType = BloomType.NONE;
147 private int blockSize;
148 private Compression.Algorithm compression = Compression.Algorithm.NONE;
149
150 private byte[] firstRow, lastRow;
151
152 private AtomicLong numSeeks = new AtomicLong();
153 private AtomicLong numKV = new AtomicLong();
154 private AtomicLong totalBytes = new AtomicLong();
155
156 private byte[] family;
157
158 private long endTime = Long.MAX_VALUE;
159
160 private SortedSet<String> keysRead = new ConcurrentSkipListSet<String>();
161 private List<StoreFile> inputStoreFiles;
162
163 public HFileReadWriteTest() {
164 conf = HBaseConfiguration.create();
165 cacheConf = new CacheConfig(conf);
166 }
167
168 @SuppressWarnings("unchecked")
169 public boolean parseOptions(String args[]) {
170
171 Options options = new Options();
172 options.addOption(OUTPUT_DIR_OPTION, true, "Output directory" +
173 Workload.MERGE.onlyUsedFor());
174 options.addOption(COMPRESSION_OPTION, true, " Compression type, one of "
175 + Arrays.toString(Compression.Algorithm.values()) +
176 Workload.MERGE.onlyUsedFor());
177 options.addOption(BLOOM_FILTER_OPTION, true, "Bloom filter type, one of "
178 + Arrays.toString(BloomType.values()) +
179 Workload.MERGE.onlyUsedFor());
180 options.addOption(BLOCK_SIZE_OPTION, true, "HFile block size" +
181 Workload.MERGE.onlyUsedFor());
182 options.addOption(DURATION_OPTION, true, "The amount of time to run the " +
183 "random read workload for" + Workload.RANDOM_READS.onlyUsedFor());
184 options.addOption(NUM_THREADS_OPTION, true, "The number of random " +
185 "reader threads" + Workload.RANDOM_READS.onlyUsedFor());
186 options.addOption(NUM_THREADS_OPTION, true, "The number of random " +
187 "reader threads" + Workload.RANDOM_READS.onlyUsedFor());
188 options.addOption(LoadTestTool.OPT_DATA_BLOCK_ENCODING, true,
189 LoadTestTool.OPT_DATA_BLOCK_ENCODING_USAGE);
190 options.addOptionGroup(Workload.getOptionGroup());
191
192 if (args.length == 0) {
193 HelpFormatter formatter = new HelpFormatter();
194 formatter.printHelp(HFileReadWriteTest.class.getSimpleName(),
195 options, true);
196 return false;
197 }
198
199 CommandLineParser parser = new PosixParser();
200 CommandLine cmdLine;
201 try {
202 cmdLine = parser.parse(options, args);
203 } catch (ParseException ex) {
204 LOG.error(ex);
205 return false;
206 }
207
208 workload = Workload.fromCmdLine(cmdLine);
209 if (workload == null)
210 return false;
211
212 inputFileNames = (List<String>) cmdLine.getArgList();
213
214 if (inputFileNames.size() == 0) {
215 LOG.error("No input file names specified");
216 return false;
217 }
218
219 if (inputFileNames.size() < workload.minNumInputFiles) {
220 LOG.error("Too few input files: at least " + workload.minNumInputFiles +
221 " required");
222 return false;
223 }
224
225 if (inputFileNames.size() > workload.maxNumInputFiles) {
226 LOG.error("Too many input files: at most " + workload.minNumInputFiles +
227 " allowed");
228 return false;
229 }
230
231 if (cmdLine.hasOption(COMPRESSION_OPTION)) {
232 compression = Compression.Algorithm.valueOf(
233 cmdLine.getOptionValue(COMPRESSION_OPTION));
234 }
235
236 if (cmdLine.hasOption(BLOOM_FILTER_OPTION)) {
237 bloomType = BloomType.valueOf(cmdLine.getOptionValue(
238 BLOOM_FILTER_OPTION));
239 }
240
241 if (cmdLine.hasOption(LoadTestTool.OPT_DATA_BLOCK_ENCODING)) {
242 dataBlockEncoding = DataBlockEncoding.valueOf(
243 cmdLine.getOptionValue(LoadTestTool.OPT_DATA_BLOCK_ENCODING));
244 }
245
246 blockSize = conf.getInt("hfile.min.blocksize.size", 65536);
247 if (cmdLine.hasOption(BLOCK_SIZE_OPTION))
248 blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION));
249
250 if (workload == Workload.MERGE) {
251 String outputDirStr = cmdLine.getOptionValue(OUTPUT_DIR_OPTION);
252 if (outputDirStr == null) {
253 LOG.error("Output directory is not specified");
254 return false;
255 }
256 outputDir = new Path(outputDirStr);
257
258 }
259
260 if (workload == Workload.RANDOM_READS) {
261 if (!requireOptions(cmdLine, new String[] { DURATION_OPTION,
262 NUM_THREADS_OPTION })) {
263 return false;
264 }
265
266 durationSec = Integer.parseInt(cmdLine.getOptionValue(DURATION_OPTION));
267 numReadThreads = Integer.parseInt(
268 cmdLine.getOptionValue(NUM_THREADS_OPTION));
269 }
270
271 Collections.sort(inputFileNames);
272
273 return true;
274 }
275
276
277 private boolean requireOptions(CommandLine cmdLine,
278 String[] requiredOptions) {
279 for (String option : requiredOptions)
280 if (!cmdLine.hasOption(option)) {
281 LOG.error("Required option -" + option + " not specified");
282 return false;
283 }
284 return true;
285 }
286
287 public boolean validateConfiguration() throws IOException {
288 fs = FileSystem.get(conf);
289
290 for (String inputFileName : inputFileNames) {
291 Path path = new Path(inputFileName);
292 if (!fs.exists(path)) {
293 LOG.error("File " + inputFileName + " does not exist");
294 return false;
295 }
296
297 if (fs.getFileStatus(path).isDir()) {
298 LOG.error(inputFileName + " is a directory");
299 return false;
300 }
301 }
302
303 if (outputDir != null &&
304 (!fs.exists(outputDir) || !fs.getFileStatus(outputDir).isDir())) {
305 LOG.error(outputDir.toString() + " does not exist or is not a " +
306 "directory");
307 return false;
308 }
309
310 return true;
311 }
312
313 public void runMergeWorkload() throws IOException {
314 long maxKeyCount = prepareForMerge();
315
316 List<StoreFileScanner> scanners =
317 StoreFileScanner.getScannersForStoreFiles(inputStoreFiles, false,
318 false);
319
320 HColumnDescriptor columnDescriptor = new HColumnDescriptor(
321 HFileReadWriteTest.class.getSimpleName());
322 columnDescriptor.setBlocksize(blockSize);
323 columnDescriptor.setBloomFilterType(bloomType);
324 columnDescriptor.setCompressionType(compression);
325 columnDescriptor.setDataBlockEncoding(dataBlockEncoding);
326 HRegionInfo regionInfo = new HRegionInfo();
327 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE_NAME));
328 HRegion region = new HRegion(outputDir, null, fs, conf, regionInfo, htd, null);
329 HStore store = new HStore(region, columnDescriptor, conf);
330
331 StoreFile.Writer writer = store.createWriterInTmp(maxKeyCount, compression, false, true);
332
333 StatisticsPrinter statsPrinter = new StatisticsPrinter();
334 statsPrinter.startThread();
335
336 try {
337 performMerge(scanners, store, writer);
338 writer.close();
339 } finally {
340 statsPrinter.requestStop();
341 }
342
343 Path resultPath = writer.getPath();
344
345 resultPath = tryUsingSimpleOutputPath(resultPath);
346
347 long fileSize = fs.getFileStatus(resultPath).getLen();
348 LOG.info("Created " + resultPath + ", size " + fileSize);
349
350 System.out.println();
351 System.out.println("HFile information for " + resultPath);
352 System.out.println();
353
354 HFilePrettyPrinter hfpp = new HFilePrettyPrinter();
355 hfpp.run(new String[] { "-m", "-f", resultPath.toString() });
356 }
357
358 private Path tryUsingSimpleOutputPath(Path resultPath) throws IOException {
359 if (inputFileNames.size() == 1) {
360
361
362
363 Path inputPath = new Path(inputFileNames.get(0));
364 Path betterOutputPath = new Path(outputDir,
365 inputPath.getName());
366 if (!fs.exists(betterOutputPath)) {
367 fs.rename(resultPath, betterOutputPath);
368 resultPath = betterOutputPath;
369 }
370 }
371 return resultPath;
372 }
373
374 private void performMerge(List<StoreFileScanner> scanners, HStore store,
375 StoreFile.Writer writer) throws IOException {
376 InternalScanner scanner = null;
377 try {
378 Scan scan = new Scan();
379
380
381 scanner = new StoreScanner(store, store.getScanInfo(), scan, scanners,
382 ScanType.COMPACT_DROP_DELETES, Long.MIN_VALUE, Long.MIN_VALUE);
383
384 ArrayList<Cell> kvs = new ArrayList<Cell>();
385
386 while (scanner.next(kvs) || kvs.size() != 0) {
387 numKV.addAndGet(kvs.size());
388 for (Cell c : kvs) {
389 KeyValue kv = KeyValueUtil.ensureKeyValue(c);
390 totalBytes.addAndGet(kv.getLength());
391 writer.append(kv);
392 }
393 kvs.clear();
394 }
395 } finally {
396 if (scanner != null)
397 scanner.close();
398 }
399 }
400
401
402
403
404
405 private long prepareForMerge() throws IOException {
406 LOG.info("Merging " + inputFileNames);
407 LOG.info("Using block size: " + blockSize);
408 inputStoreFiles = new ArrayList<StoreFile>();
409
410 long maxKeyCount = 0;
411 for (String fileName : inputFileNames) {
412 Path filePath = new Path(fileName);
413
414
415 StoreFile sf = openStoreFile(filePath, false);
416 sf.createReader();
417 inputStoreFiles.add(sf);
418
419 StoreFile.Reader r = sf.getReader();
420 if (r != null) {
421 long keyCount = r.getFilterEntries();
422 maxKeyCount += keyCount;
423 LOG.info("Compacting: " + sf + "; keyCount = " + keyCount
424 + "; Bloom Type = " + r.getBloomFilterType().toString()
425 + "; Size = " + StringUtils.humanReadableInt(r.length()));
426 }
427 }
428 return maxKeyCount;
429 }
430
431 public HFile.Reader[] getHFileReaders() {
432 HFile.Reader readers[] = new HFile.Reader[inputStoreFiles.size()];
433 for (int i = 0; i < inputStoreFiles.size(); ++i)
434 readers[i] = inputStoreFiles.get(i).getReader().getHFileReader();
435 return readers;
436 }
437
438 private StoreFile openStoreFile(Path filePath, boolean blockCache)
439 throws IOException {
440
441
442 return new StoreFile(fs, filePath, conf, cacheConf,
443 BloomType.ROWCOL);
444 }
445
446 public static int charToHex(int c) {
447 if ('0' <= c && c <= '9')
448 return c - '0';
449 if ('a' <= c && c <= 'f')
450 return 10 + c - 'a';
451 return -1;
452 }
453
454 public static int hexToChar(int h) {
455 h &= 0xff;
456 if (0 <= h && h <= 9)
457 return '0' + h;
458 if (10 <= h && h <= 15)
459 return 'a' + h - 10;
460 return -1;
461 }
462
463 public static byte[] createRandomRow(Random rand, byte[] first, byte[] last)
464 {
465 int resultLen = Math.max(first.length, last.length);
466 int minLen = Math.min(first.length, last.length);
467 byte[] result = new byte[resultLen];
468 boolean greaterThanFirst = false;
469 boolean lessThanLast = false;
470
471 for (int i = 0; i < resultLen; ++i) {
472
473
474 boolean isHex = i < minLen && charToHex(first[i]) != -1
475 && charToHex(last[i]) != -1;
476
477
478
479 int low = greaterThanFirst || i >= first.length ? 0 : first[i] & 0xff;
480
481
482
483 int high = lessThanLast || i >= last.length ? 0xff : last[i] & 0xff;
484
485
486
487
488
489
490 int r;
491 if (isHex) {
492
493 if (low < '0')
494 low = '0';
495
496 if (high > 'f')
497 high = 'f';
498
499 int lowHex = charToHex(low);
500 int highHex = charToHex(high);
501 r = hexToChar(lowHex + rand.nextInt(highHex - lowHex + 1));
502 } else {
503 r = low + rand.nextInt(high - low + 1);
504 }
505
506 if (r > low)
507 greaterThanFirst = true;
508
509 if (r < high)
510 lessThanLast = true;
511
512 result[i] = (byte) r;
513 }
514
515 if (Bytes.compareTo(result, first) < 0) {
516 throw new IllegalStateException("Generated key " +
517 Bytes.toStringBinary(result) + " is less than the first key " +
518 Bytes.toStringBinary(first));
519 }
520
521 if (Bytes.compareTo(result, last) > 0) {
522 throw new IllegalStateException("Generated key " +
523 Bytes.toStringBinary(result) + " is greater than te last key " +
524 Bytes.toStringBinary(last));
525 }
526
527 return result;
528 }
529
530 private static byte[] createRandomQualifier(Random rand) {
531 byte[] q = new byte[10 + rand.nextInt(30)];
532 rand.nextBytes(q);
533 return q;
534 }
535
536 private class RandomReader implements Callable<Boolean> {
537
538 private int readerId;
539 private StoreFile.Reader reader;
540 private boolean pread;
541
542 public RandomReader(int readerId, StoreFile.Reader reader,
543 boolean pread)
544 {
545 this.readerId = readerId;
546 this.reader = reader;
547 this.pread = pread;
548 }
549
550 @Override
551 public Boolean call() throws Exception {
552 Thread.currentThread().setName("reader " + readerId);
553 Random rand = new Random();
554 StoreFileScanner scanner = reader.getStoreFileScanner(true, pread);
555
556 while (System.currentTimeMillis() < endTime) {
557 byte[] row = createRandomRow(rand, firstRow, lastRow);
558 KeyValue kvToSeek = new KeyValue(row, family,
559 createRandomQualifier(rand));
560 if (rand.nextDouble() < 0.0001) {
561 LOG.info("kvToSeek=" + kvToSeek);
562 }
563 boolean seekResult;
564 try {
565 seekResult = scanner.seek(kvToSeek);
566 } catch (IOException ex) {
567 throw new IOException("Seek failed for key " + kvToSeek + ", pread="
568 + pread, ex);
569 }
570 numSeeks.incrementAndGet();
571 if (!seekResult) {
572 error("Seek returned false for row " + Bytes.toStringBinary(row));
573 return false;
574 }
575 for (int i = 0; i < rand.nextInt(10) + 1; ++i) {
576 KeyValue kv = scanner.next();
577 numKV.incrementAndGet();
578 if (i == 0 && kv == null) {
579 error("scanner.next() returned null at the first iteration for " +
580 "row " + Bytes.toStringBinary(row));
581 return false;
582 }
583 if (kv == null)
584 break;
585
586 String keyHashStr = MD5Hash.getMD5AsHex(kv.getKey());
587 keysRead.add(keyHashStr);
588 totalBytes.addAndGet(kv.getLength());
589 }
590 }
591
592 return true;
593 }
594
595 private void error(String msg) {
596 LOG.error("error in reader " + readerId + " (pread=" + pread + "): "
597 + msg);
598 }
599
600 }
601
602 private class StatisticsPrinter implements Callable<Boolean> {
603
604 private volatile boolean stopRequested;
605 private volatile Thread thread;
606 private long totalSeekAndReads, totalPositionalReads;
607
608
609
610
611 public void startThread() {
612 new Thread() {
613 @Override
614 public void run() {
615 try {
616 call();
617 } catch (Exception e) {
618 LOG.error(e);
619 }
620 }
621 }.start();
622 }
623
624 @Override
625 public Boolean call() throws Exception {
626 LOG.info("Starting statistics printer");
627 thread = Thread.currentThread();
628 thread.setName(StatisticsPrinter.class.getSimpleName());
629 long startTime = System.currentTimeMillis();
630 long curTime;
631 while ((curTime = System.currentTimeMillis()) < endTime &&
632 !stopRequested) {
633 long elapsedTime = curTime - startTime;
634 printStats(elapsedTime);
635 try {
636 Thread.sleep(1000 - elapsedTime % 1000);
637 } catch (InterruptedException iex) {
638 Thread.currentThread().interrupt();
639 if (stopRequested)
640 break;
641 }
642 }
643 printStats(curTime - startTime);
644 LOG.info("Stopping statistics printer");
645 return true;
646 }
647
648 private void printStats(long elapsedTime) {
649 long numSeeksL = numSeeks.get();
650 double timeSec = elapsedTime / 1000.0;
651 double seekPerSec = numSeeksL / timeSec;
652 long kvCount = numKV.get();
653 double kvPerSec = kvCount / timeSec;
654 long bytes = totalBytes.get();
655 double bytesPerSec = bytes / timeSec;
656
657
658
659
660
661 totalSeekAndReads += HFile.getReadOps();
662 totalPositionalReads += HFile.getPreadOps();
663 long totalBlocksRead = totalSeekAndReads + totalPositionalReads;
664
665 double blkReadPerSec = totalBlocksRead / timeSec;
666
667 double seekReadPerSec = totalSeekAndReads / timeSec;
668 double preadPerSec = totalPositionalReads / timeSec;
669
670 boolean isRead = workload == Workload.RANDOM_READS;
671
672 StringBuilder sb = new StringBuilder();
673 sb.append("Time: " + (long) timeSec + " sec");
674 if (isRead)
675 sb.append(", seek/sec: " + (long) seekPerSec);
676 sb.append(", kv/sec: " + (long) kvPerSec);
677 sb.append(", bytes/sec: " + (long) bytesPerSec);
678 sb.append(", blk/sec: " + (long) blkReadPerSec);
679 sb.append(", total KV: " + numKV);
680 sb.append(", total bytes: " + totalBytes);
681 sb.append(", total blk: " + totalBlocksRead);
682
683 sb.append(", seekRead/sec: " + (long) seekReadPerSec);
684 sb.append(", pread/sec: " + (long) preadPerSec);
685
686 if (isRead)
687 sb.append(", unique keys: " + (long) keysRead.size());
688
689 LOG.info(sb.toString());
690 }
691
692 public void requestStop() {
693 stopRequested = true;
694 if (thread != null)
695 thread.interrupt();
696 }
697
698 }
699
700 public boolean runRandomReadWorkload() throws IOException {
701 if (inputFileNames.size() != 1) {
702 throw new IOException("Need exactly one input file for random reads: " +
703 inputFileNames);
704 }
705
706 Path inputPath = new Path(inputFileNames.get(0));
707
708
709 StoreFile storeFile = openStoreFile(inputPath, true);
710
711 StoreFile.Reader reader = storeFile.createReader();
712
713 LOG.info("First key: " + Bytes.toStringBinary(reader.getFirstKey()));
714 LOG.info("Last key: " + Bytes.toStringBinary(reader.getLastKey()));
715
716 KeyValue firstKV = KeyValue.createKeyValueFromKey(reader.getFirstKey());
717 firstRow = firstKV.getRow();
718
719 KeyValue lastKV = KeyValue.createKeyValueFromKey(reader.getLastKey());
720 lastRow = lastKV.getRow();
721
722 byte[] family = firstKV.getFamily();
723 if (!Bytes.equals(family, lastKV.getFamily())) {
724 LOG.error("First and last key have different families: "
725 + Bytes.toStringBinary(family) + " and "
726 + Bytes.toStringBinary(lastKV.getFamily()));
727 return false;
728 }
729
730 if (Bytes.equals(firstRow, lastRow)) {
731 LOG.error("First and last row are the same, cannot run read workload: " +
732 "firstRow=" + Bytes.toStringBinary(firstRow) + ", " +
733 "lastRow=" + Bytes.toStringBinary(lastRow));
734 return false;
735 }
736
737 ExecutorService exec = Executors.newFixedThreadPool(numReadThreads + 1);
738 int numCompleted = 0;
739 int numFailed = 0;
740 try {
741 ExecutorCompletionService<Boolean> ecs =
742 new ExecutorCompletionService<Boolean>(exec);
743 endTime = System.currentTimeMillis() + 1000 * durationSec;
744 boolean pread = true;
745 for (int i = 0; i < numReadThreads; ++i)
746 ecs.submit(new RandomReader(i, reader, pread));
747 ecs.submit(new StatisticsPrinter());
748 Future<Boolean> result;
749 while (true) {
750 try {
751 result = ecs.poll(endTime + 1000 - System.currentTimeMillis(),
752 TimeUnit.MILLISECONDS);
753 if (result == null)
754 break;
755 try {
756 if (result.get()) {
757 ++numCompleted;
758 } else {
759 ++numFailed;
760 }
761 } catch (ExecutionException e) {
762 LOG.error("Worker thread failure", e.getCause());
763 ++numFailed;
764 }
765 } catch (InterruptedException ex) {
766 LOG.error("Interrupted after " + numCompleted +
767 " workers completed");
768 Thread.currentThread().interrupt();
769 continue;
770 }
771
772 }
773 } finally {
774 storeFile.closeReader(true);
775 exec.shutdown();
776
777 BlockCache c = cacheConf.getBlockCache();
778 if (c != null) {
779 c.shutdown();
780 }
781 }
782 LOG.info("Worker threads completed: " + numCompleted);
783 LOG.info("Worker threads failed: " + numFailed);
784 return true;
785 }
786
787 public boolean run() throws IOException {
788 LOG.info("Workload: " + workload);
789 switch (workload) {
790 case MERGE:
791 runMergeWorkload();
792 break;
793 case RANDOM_READS:
794 return runRandomReadWorkload();
795 default:
796 LOG.error("Unknown workload: " + workload);
797 return false;
798 }
799
800 return true;
801 }
802
803 private static void failure() {
804 System.exit(1);
805 }
806
807 public static void main(String[] args) {
808 HFileReadWriteTest app = new HFileReadWriteTest();
809 if (!app.parseOptions(args))
810 failure();
811
812 try {
813 if (!app.validateConfiguration() ||
814 !app.run())
815 failure();
816 } catch (IOException ex) {
817 LOG.error(ex);
818 failure();
819 }
820 }
821
822 }