001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.regionserver; 019 020import java.io.IOException; 021import java.util.Arrays; 022import java.util.Random; 023import java.util.concurrent.ThreadLocalRandom; 024import org.apache.hadoop.conf.Configuration; 025import org.apache.hadoop.fs.FileSystem; 026import org.apache.hadoop.fs.Path; 027import org.apache.hadoop.hbase.HBaseConfiguration; 028import org.apache.hadoop.hbase.HConstants; 029import org.apache.hadoop.hbase.KeyValue; 030import org.apache.hadoop.hbase.io.compress.Compression; 031import org.apache.hadoop.hbase.io.hfile.CacheConfig; 032import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex; 033import org.apache.hadoop.hbase.io.hfile.HFileContext; 034import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder; 035import org.apache.hadoop.hbase.util.BloomFilterFactory; 036import org.apache.hadoop.hbase.util.BloomFilterUtil; 037import org.apache.hadoop.io.BytesWritable; 038import org.slf4j.Logger; 039import org.slf4j.LoggerFactory; 040 041import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 042import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; 043import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; 044import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 045import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 046import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser; 047 048/** 049 * Creates an HFile with random key/value pairs. 050 */ 051public class CreateRandomStoreFile { 052 053 /** 054 * As much as this number of bytes can be added or subtracted from key/value lengths. 055 */ 056 private static final int LEN_VARIATION = 5; 057 058 private static final Logger LOG = LoggerFactory.getLogger(CreateRandomStoreFile.class); 059 private static final String OUTPUT_DIR_OPTION = "o"; 060 private static final String NUM_KV_OPTION = "n"; 061 private static final String HFILE_VERSION_OPTION = "h"; 062 private static final String KEY_SIZE_OPTION = "k"; 063 private static final String VALUE_SIZE_OPTION = "v"; 064 private static final String COMPRESSION_OPTION = "c"; 065 private static final String BLOOM_FILTER_OPTION = "bf"; 066 private static final String BLOOM_FILTER_PARAM_OPTION = "bfp"; 067 private static final String BLOCK_SIZE_OPTION = "bs"; 068 private static final String BLOOM_BLOCK_SIZE_OPTION = "bfbs"; 069 private static final String INDEX_BLOCK_SIZE_OPTION = "ibs"; 070 071 /** The exit code this command-line tool returns on failure */ 072 private static final int EXIT_FAILURE = 1; 073 074 /** The number of valid key types in a store file */ 075 private static final int NUM_VALID_KEY_TYPES = KeyValue.Type.values().length - 2; 076 077 private Options options = new Options(); 078 079 private int keyPrefixLen, keyLen, rowLen, cfLen, valueLen; 080 081 /** 082 * Runs the tools. 083 * @param args command-line arguments 084 * @return true in case of success 085 */ 086 public boolean run(String[] args) throws IOException { 087 options.addOption(OUTPUT_DIR_OPTION, "output_dir", true, "Output directory"); 088 options.addOption(NUM_KV_OPTION, "num_kv", true, "Number of key/value pairs"); 089 options.addOption(KEY_SIZE_OPTION, "key_size", true, "Average key size"); 090 options.addOption(VALUE_SIZE_OPTION, "value_size", true, "Average value size"); 091 options.addOption(HFILE_VERSION_OPTION, "hfile_version", true, "HFile version to create"); 092 options.addOption(COMPRESSION_OPTION, "compression", true, 093 " Compression type, one of " + Arrays.toString(Compression.Algorithm.values())); 094 options.addOption(BLOOM_FILTER_OPTION, "bloom_filter", true, 095 "Bloom filter type, one of " + Arrays.toString(BloomType.values())); 096 options.addOption(BLOOM_FILTER_PARAM_OPTION, "bloom_param", true, 097 "the parameter of the bloom filter"); 098 options.addOption(BLOCK_SIZE_OPTION, "block_size", true, "HFile block size"); 099 options.addOption(BLOOM_BLOCK_SIZE_OPTION, "bloom_block_size", true, 100 "Compound Bloom filters block size"); 101 options.addOption(INDEX_BLOCK_SIZE_OPTION, "index_block_size", true, "Index block size"); 102 103 if (args.length == 0) { 104 HelpFormatter formatter = new HelpFormatter(); 105 formatter.printHelp(CreateRandomStoreFile.class.getSimpleName(), options, true); 106 return false; 107 } 108 109 CommandLineParser parser = new PosixParser(); 110 CommandLine cmdLine; 111 try { 112 cmdLine = parser.parse(options, args); 113 } catch (ParseException ex) { 114 LOG.error(ex.toString(), ex); 115 return false; 116 } 117 118 if (!cmdLine.hasOption(OUTPUT_DIR_OPTION)) { 119 LOG.error("Output directory is not specified"); 120 return false; 121 } 122 123 if (!cmdLine.hasOption(NUM_KV_OPTION)) { 124 LOG.error("The number of keys/values not specified"); 125 return false; 126 } 127 128 if (!cmdLine.hasOption(KEY_SIZE_OPTION)) { 129 LOG.error("Key size is not specified"); 130 return false; 131 } 132 133 if (!cmdLine.hasOption(VALUE_SIZE_OPTION)) { 134 LOG.error("Value size not specified"); 135 return false; 136 } 137 138 Configuration conf = HBaseConfiguration.create(); 139 140 Path outputDir = new Path(cmdLine.getOptionValue(OUTPUT_DIR_OPTION)); 141 142 long numKV = Long.parseLong(cmdLine.getOptionValue(NUM_KV_OPTION)); 143 configureKeyValue(numKV, Integer.parseInt(cmdLine.getOptionValue(KEY_SIZE_OPTION)), 144 Integer.parseInt(cmdLine.getOptionValue(VALUE_SIZE_OPTION))); 145 146 FileSystem fs = FileSystem.get(conf); 147 148 Compression.Algorithm compr = Compression.Algorithm.NONE; 149 if (cmdLine.hasOption(COMPRESSION_OPTION)) { 150 compr = Compression.Algorithm.valueOf(cmdLine.getOptionValue(COMPRESSION_OPTION)); 151 } 152 153 BloomType bloomType = BloomType.NONE; 154 if (cmdLine.hasOption(BLOOM_FILTER_OPTION)) { 155 bloomType = BloomType.valueOf(cmdLine.getOptionValue(BLOOM_FILTER_OPTION)); 156 } 157 158 if (bloomType == BloomType.ROWPREFIX_FIXED_LENGTH) { 159 if (!cmdLine.hasOption(BLOOM_FILTER_PARAM_OPTION)) { 160 LOG.error("the parameter of bloom filter is not specified"); 161 return false; 162 } else { 163 conf.set(BloomFilterUtil.PREFIX_LENGTH_KEY, 164 cmdLine.getOptionValue(BLOOM_FILTER_PARAM_OPTION)); 165 } 166 } 167 168 int blockSize = HConstants.DEFAULT_BLOCKSIZE; 169 if (cmdLine.hasOption(BLOCK_SIZE_OPTION)) 170 blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION)); 171 172 if (cmdLine.hasOption(BLOOM_BLOCK_SIZE_OPTION)) { 173 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, 174 Integer.valueOf(cmdLine.getOptionValue(BLOOM_BLOCK_SIZE_OPTION))); 175 } 176 177 if (cmdLine.hasOption(INDEX_BLOCK_SIZE_OPTION)) { 178 conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, 179 Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION))); 180 } 181 182 HFileContext meta = 183 new HFileContextBuilder().withCompression(compr).withBlockSize(blockSize).build(); 184 StoreFileWriter sfw = 185 new StoreFileWriter.Builder(conf, new CacheConfig(conf), fs).withOutputDir(outputDir) 186 .withBloomType(bloomType).withMaxKeyCount(numKV).withFileContext(meta).build(); 187 188 LOG.info("Writing " + numKV + " key/value pairs"); 189 for (long i = 0; i < numKV; ++i) { 190 sfw.append(generateKeyValue(i)); 191 } 192 193 int numMetaBlocks = ThreadLocalRandom.current().nextInt(10) + 1; 194 LOG.info("Writing " + numMetaBlocks + " meta blocks"); 195 for (int metaI = 0; metaI < numMetaBlocks; ++metaI) { 196 sfw.getLiveFileWriter().appendMetaBlock(generateString(), new BytesWritable(generateValue())); 197 } 198 sfw.close(); 199 200 Path storeFilePath = sfw.getPath(); 201 long fileSize = fs.getFileStatus(storeFilePath).getLen(); 202 LOG.info("Created {}, {} bytes, compression={}", storeFilePath, fileSize, compr.toString()); 203 204 return true; 205 } 206 207 private void configureKeyValue(long numKV, int keyLen, int valueLen) { 208 numKV = Math.abs(numKV); 209 keyLen = Math.abs(keyLen); 210 keyPrefixLen = 0; 211 while (numKV != 0) { 212 numKV >>>= 8; 213 ++keyPrefixLen; 214 } 215 216 this.keyLen = Math.max(keyPrefixLen, keyLen); 217 this.valueLen = valueLen; 218 219 // Arbitrarily split the key into row, column family, and qualifier. 220 rowLen = keyPrefixLen / 3; 221 cfLen = keyPrefixLen / 4; 222 } 223 224 private int nextInRange(int range) { 225 return ThreadLocalRandom.current().nextInt(2 * range + 1) - range; 226 } 227 228 public KeyValue generateKeyValue(long i) { 229 byte[] k = generateKey(i); 230 byte[] v = generateValue(); 231 Random rand = ThreadLocalRandom.current(); 232 return new KeyValue(k, 0, rowLen, k, rowLen, cfLen, k, rowLen + cfLen, 233 k.length - rowLen - cfLen, rand.nextLong(), generateKeyType(rand), v, 0, v.length); 234 } 235 236 public static KeyValue.Type generateKeyType(Random rand) { 237 if (rand.nextBoolean()) { 238 // Let's make half of KVs puts. 239 return KeyValue.Type.Put; 240 } else { 241 KeyValue.Type keyType = KeyValue.Type.values()[1 + rand.nextInt(NUM_VALID_KEY_TYPES)]; 242 if (keyType == KeyValue.Type.Minimum || keyType == KeyValue.Type.Maximum) { 243 throw new RuntimeException("Generated an invalid key type: " + keyType + ". " 244 + "Probably the layout of KeyValue.Type has changed."); 245 } 246 return keyType; 247 } 248 } 249 250 private String generateString() { 251 Random rand = ThreadLocalRandom.current(); 252 StringBuilder sb = new StringBuilder(); 253 for (int i = 0; i < rand.nextInt(10); ++i) { 254 sb.append((char) ('A' + rand.nextInt(26))); 255 } 256 return sb.toString(); 257 } 258 259 private byte[] generateKey(long i) { 260 Random rand = ThreadLocalRandom.current(); 261 byte[] k = new byte[Math.max(keyPrefixLen, keyLen + nextInRange(LEN_VARIATION))]; 262 for (int pos = keyPrefixLen - 1; pos >= 0; --pos) { 263 k[pos] = (byte) (i & 0xFF); 264 i >>>= 8; 265 } 266 for (int pos = keyPrefixLen; pos < k.length; ++pos) { 267 k[pos] = (byte) rand.nextInt(256); 268 } 269 return k; 270 } 271 272 private byte[] generateValue() { 273 Random rand = ThreadLocalRandom.current(); 274 byte[] v = new byte[Math.max(1, valueLen + nextInRange(LEN_VARIATION))]; 275 for (int i = 0; i < v.length; ++i) { 276 v[i] = (byte) rand.nextInt(256); 277 } 278 return v; 279 } 280 281 public static void main(String[] args) { 282 CreateRandomStoreFile app = new CreateRandomStoreFile(); 283 try { 284 if (!app.run(args)) System.exit(EXIT_FAILURE); 285 } catch (IOException ex) { 286 LOG.error(ex.toString(), ex); 287 System.exit(EXIT_FAILURE); 288 } 289 290 } 291 292}