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