commit aee202f1ec19551de9e7387382de071608f79e89 Author: nspiegelberg Date: 57 seconds ago HBASE-4772 Utility to Create StoreFiles for Test Purposes diff --git a/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java b/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java index 1b183af..42ea29b 100644 --- a/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java +++ b/src/test/java/org/apache/hadoop/hbase/regionserver/CreateRandomStoreFile.java @@ -36,8 +36,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.io.hfile.CacheConfig; import org.apache.hadoop.hbase.io.hfile.Compression; +import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.util.BloomFilterFactory; import org.apache.hadoop.io.BytesWritable; /** @@ -55,10 +58,14 @@ public class CreateRandomStoreFile { LogFactory.getLog(CreateRandomStoreFile.class); private static final String OUTPUT_DIR_OPTION = "o"; private static final String NUM_KV_OPTION = "n"; + private static final String HFILE_VERSION_OPTION = "h"; private static final String KEY_SIZE_OPTION = "k"; private static final String VALUE_SIZE_OPTION = "v"; private static final String COMPRESSION_OPTION = "c"; - private static final String BLOOM_FILTER_OPTION = "b"; + private static final String BLOOM_FILTER_OPTION = "bf"; + private static final String BLOCK_SIZE_OPTION = "bs"; + private static final String BLOOM_BLOCK_SIZE_OPTION = "bfbs"; + private static final String INDEX_BLOCK_SIZE_OPTION = "ibs"; /** The exit code this command-line tool returns on failure */ private static final int EXIT_FAILURE = 1; @@ -87,12 +94,20 @@ public class CreateRandomStoreFile { options.addOption(KEY_SIZE_OPTION, "key_size", true, "Average key size"); options.addOption(VALUE_SIZE_OPTION, "value_size", true, "Average value size"); + options.addOption(HFILE_VERSION_OPTION, "hfile_version", true, + "HFile version to create"); options.addOption(COMPRESSION_OPTION, "compression", true, " Compression type, one of " + Arrays.toString(Compression.Algorithm.values())); options.addOption(BLOOM_FILTER_OPTION, "bloom_filter", true, "Bloom filter type, one of " + Arrays.toString(StoreFile.BloomType.values())); + options.addOption(BLOCK_SIZE_OPTION, "block_size", true, + "HFile block size"); + options.addOption(BLOOM_BLOCK_SIZE_OPTION, "bloom_block_size", true, + "Compound Bloom filters block size"); + options.addOption(INDEX_BLOCK_SIZE_OPTION, "index_block_size", true, + "Index block size"); if (args.length == 0) { HelpFormatter formatter = new HelpFormatter(); @@ -153,9 +168,23 @@ public class CreateRandomStoreFile { BLOOM_FILTER_OPTION)); } - StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir, - conf.getInt("hfile.min.blocksize.size", 65536), compr, - KeyValue.COMPARATOR, conf, bloomType, numKV, false); + int blockSize = conf.getInt("hfile.min.blocksize.size", 65536); + if (cmdLine.hasOption(BLOCK_SIZE_OPTION)) + blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION)); + + if (cmdLine.hasOption(BLOOM_BLOCK_SIZE_OPTION)) { + conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, + Integer.valueOf(cmdLine.getOptionValue(BLOOM_BLOCK_SIZE_OPTION))); + } + + if (cmdLine.hasOption(INDEX_BLOCK_SIZE_OPTION)) { + conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, + Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION))); + } + + StoreFile.Writer sfw = StoreFile.createWriter(fs, outputDir, blockSize, + compr, KeyValue.COMPARATOR, conf, new CacheConfig(conf), bloomType, + numKV); rand = new Random(); LOG.info("Writing " + numKV + " key/value pairs"); @@ -208,11 +237,11 @@ public class CreateRandomStoreFile { k, rowLen, cfLen, k, rowLen + cfLen, k.length - rowLen - cfLen, rand.nextLong(), - generateKeyType(), + generateKeyType(rand), v, 0, v.length); } - private KeyValue.Type generateKeyType() { + public static KeyValue.Type generateKeyType(Random rand) { if (rand.nextBoolean()) { // Let's make half of KVs puts. return KeyValue.Type.Put;