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.io.BytesWritable; 038import org.slf4j.Logger; 039import org.slf4j.LoggerFactory; 040import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine; 041import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser; 042import org.apache.hbase.thirdparty.org.apache.commons.cli.HelpFormatter; 043import org.apache.hbase.thirdparty.org.apache.commons.cli.Options; 044import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException; 045import org.apache.hbase.thirdparty.org.apache.commons.cli.PosixParser; 046 047/** 048 * Creates an HFile with random key/value pairs. 049 */ 050public class CreateRandomStoreFile { 051 052 /** 053 * As much as this number of bytes can be added or subtracted from key/value 054 * lengths. 055 */ 056 private static final int LEN_VARIATION = 5; 057 058 private static final Logger LOG = 059 LoggerFactory.getLogger(CreateRandomStoreFile.class); 060 private static final String OUTPUT_DIR_OPTION = "o"; 061 private static final String NUM_KV_OPTION = "n"; 062 private static final String HFILE_VERSION_OPTION = "h"; 063 private static final String KEY_SIZE_OPTION = "k"; 064 private static final String VALUE_SIZE_OPTION = "v"; 065 private static final String COMPRESSION_OPTION = "c"; 066 private static final String BLOOM_FILTER_OPTION = "bf"; 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 = 076 KeyValue.Type.values().length - 2; 077 078 private Options options = new Options(); 079 080 private int keyPrefixLen, keyLen, rowLen, cfLen, valueLen; 081 private Random rand; 082 083 /** 084 * Runs the tools. 085 * 086 * @param args command-line arguments 087 * @return true in case of success 088 * @throws IOException 089 */ 090 public boolean run(String[] args) throws IOException { 091 options.addOption(OUTPUT_DIR_OPTION, "output_dir", true, 092 "Output directory"); 093 options.addOption(NUM_KV_OPTION, "num_kv", true, 094 "Number of key/value pairs"); 095 options.addOption(KEY_SIZE_OPTION, "key_size", true, "Average key size"); 096 options.addOption(VALUE_SIZE_OPTION, "value_size", true, 097 "Average value size"); 098 options.addOption(HFILE_VERSION_OPTION, "hfile_version", true, 099 "HFile version to create"); 100 options.addOption(COMPRESSION_OPTION, "compression", true, 101 " Compression type, one of " 102 + Arrays.toString(Compression.Algorithm.values())); 103 options.addOption(BLOOM_FILTER_OPTION, "bloom_filter", true, 104 "Bloom filter type, one of " 105 + Arrays.toString(BloomType.values())); 106 options.addOption(BLOCK_SIZE_OPTION, "block_size", true, 107 "HFile block size"); 108 options.addOption(BLOOM_BLOCK_SIZE_OPTION, "bloom_block_size", true, 109 "Compound Bloom filters block size"); 110 options.addOption(INDEX_BLOCK_SIZE_OPTION, "index_block_size", true, 111 "Index block size"); 112 113 if (args.length == 0) { 114 HelpFormatter formatter = new HelpFormatter(); 115 formatter.printHelp(CreateRandomStoreFile.class.getSimpleName(), options, 116 true); 117 return false; 118 } 119 120 CommandLineParser parser = new PosixParser(); 121 CommandLine cmdLine; 122 try { 123 cmdLine = parser.parse(options, args); 124 } catch (ParseException ex) { 125 LOG.error(ex.toString(), ex); 126 return false; 127 } 128 129 if (!cmdLine.hasOption(OUTPUT_DIR_OPTION)) { 130 LOG.error("Output directory is not specified"); 131 return false; 132 } 133 134 if (!cmdLine.hasOption(NUM_KV_OPTION)) { 135 LOG.error("The number of keys/values not specified"); 136 return false; 137 } 138 139 if (!cmdLine.hasOption(KEY_SIZE_OPTION)) { 140 LOG.error("Key size is not specified"); 141 return false; 142 } 143 144 if (!cmdLine.hasOption(VALUE_SIZE_OPTION)) { 145 LOG.error("Value size not specified"); 146 return false; 147 } 148 149 Configuration conf = HBaseConfiguration.create(); 150 151 Path outputDir = new Path(cmdLine.getOptionValue(OUTPUT_DIR_OPTION)); 152 153 long numKV = Long.parseLong(cmdLine.getOptionValue(NUM_KV_OPTION)); 154 configureKeyValue(numKV, 155 Integer.parseInt(cmdLine.getOptionValue(KEY_SIZE_OPTION)), 156 Integer.parseInt(cmdLine.getOptionValue(VALUE_SIZE_OPTION))); 157 158 FileSystem fs = FileSystem.get(conf); 159 160 Compression.Algorithm compr = Compression.Algorithm.NONE; 161 if (cmdLine.hasOption(COMPRESSION_OPTION)) { 162 compr = Compression.Algorithm.valueOf( 163 cmdLine.getOptionValue(COMPRESSION_OPTION)); 164 } 165 166 BloomType bloomType = BloomType.NONE; 167 if (cmdLine.hasOption(BLOOM_FILTER_OPTION)) { 168 bloomType = BloomType.valueOf(cmdLine.getOptionValue( 169 BLOOM_FILTER_OPTION)); 170 } 171 172 int blockSize = HConstants.DEFAULT_BLOCKSIZE; 173 if (cmdLine.hasOption(BLOCK_SIZE_OPTION)) 174 blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION)); 175 176 if (cmdLine.hasOption(BLOOM_BLOCK_SIZE_OPTION)) { 177 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE, 178 Integer.valueOf(cmdLine.getOptionValue(BLOOM_BLOCK_SIZE_OPTION))); 179 } 180 181 if (cmdLine.hasOption(INDEX_BLOCK_SIZE_OPTION)) { 182 conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY, 183 Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION))); 184 } 185 186 HFileContext meta = new HFileContextBuilder().withCompression(compr) 187 .withBlockSize(blockSize).build(); 188 StoreFileWriter sfw = new StoreFileWriter.Builder(conf, 189 new CacheConfig(conf), fs) 190 .withOutputDir(outputDir) 191 .withBloomType(bloomType) 192 .withMaxKeyCount(numKV) 193 .withFileContext(meta) 194 .build(); 195 196 rand = new Random(); 197 LOG.info("Writing " + numKV + " key/value pairs"); 198 for (long i = 0; i < numKV; ++i) { 199 sfw.append(generateKeyValue(i)); 200 } 201 202 int numMetaBlocks = rand.nextInt(10) + 1; 203 LOG.info("Writing " + numMetaBlocks + " meta blocks"); 204 for (int metaI = 0; metaI < numMetaBlocks; ++metaI) { 205 sfw.getHFileWriter().appendMetaBlock(generateString(), 206 new BytesWritable(generateValue())); 207 } 208 sfw.close(); 209 210 Path storeFilePath = sfw.getPath(); 211 long fileSize = fs.getFileStatus(storeFilePath).getLen(); 212 LOG.info("Created {}, {} bytes, compression={}", storeFilePath, fileSize, compr.toString()); 213 214 return true; 215 } 216 217 private void configureKeyValue(long numKV, int keyLen, int valueLen) { 218 numKV = Math.abs(numKV); 219 keyLen = Math.abs(keyLen); 220 keyPrefixLen = 0; 221 while (numKV != 0) { 222 numKV >>>= 8; 223 ++keyPrefixLen; 224 } 225 226 this.keyLen = Math.max(keyPrefixLen, keyLen); 227 this.valueLen = valueLen; 228 229 // Arbitrarily split the key into row, column family, and qualifier. 230 rowLen = keyPrefixLen / 3; 231 cfLen = keyPrefixLen / 4; 232 } 233 234 private int nextInRange(int range) { 235 return rand.nextInt(2 * range + 1) - range; 236 } 237 238 public KeyValue generateKeyValue(long i) { 239 byte[] k = generateKey(i); 240 byte[] v = generateValue(); 241 242 return new KeyValue( 243 k, 0, rowLen, 244 k, rowLen, cfLen, 245 k, rowLen + cfLen, k.length - rowLen - cfLen, 246 rand.nextLong(), 247 generateKeyType(rand), 248 v, 0, v.length); 249 } 250 251 public static KeyValue.Type generateKeyType(Random rand) { 252 if (rand.nextBoolean()) { 253 // Let's make half of KVs puts. 254 return KeyValue.Type.Put; 255 } else { 256 KeyValue.Type keyType = 257 KeyValue.Type.values()[1 + rand.nextInt(NUM_VALID_KEY_TYPES)]; 258 if (keyType == KeyValue.Type.Minimum || keyType == KeyValue.Type.Maximum) 259 { 260 throw new RuntimeException("Generated an invalid key type: " + keyType 261 + ". " + "Probably the layout of KeyValue.Type has changed."); 262 } 263 return keyType; 264 } 265 } 266 267 private String generateString() { 268 StringBuilder sb = new StringBuilder(); 269 for (int i = 0; i < rand.nextInt(10); ++i) { 270 sb.append((char) ('A' + rand.nextInt(26))); 271 } 272 return sb.toString(); 273 } 274 275 private byte[] generateKey(long i) { 276 byte[] k = new byte[Math.max(keyPrefixLen, keyLen 277 + nextInRange(LEN_VARIATION))]; 278 for (int pos = keyPrefixLen - 1; pos >= 0; --pos) { 279 k[pos] = (byte) (i & 0xFF); 280 i >>>= 8; 281 } 282 for (int pos = keyPrefixLen; pos < k.length; ++pos) { 283 k[pos] = (byte) rand.nextInt(256); 284 } 285 return k; 286 } 287 288 private byte[] generateValue() { 289 byte[] v = new byte[Math.max(1, valueLen + nextInRange(LEN_VARIATION))]; 290 for (int i = 0; i < v.length; ++i) { 291 v[i] = (byte) rand.nextInt(256); 292 } 293 return v; 294 } 295 296 public static void main(String[] args) { 297 CreateRandomStoreFile app = new CreateRandomStoreFile(); 298 try { 299 if (!app.run(args)) 300 System.exit(EXIT_FAILURE); 301 } catch (IOException ex) { 302 LOG.error(ex.toString(), ex); 303 System.exit(EXIT_FAILURE); 304 } 305 306 } 307 308}