View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  import java.util.Arrays;
23  import java.util.Random;
24  
25  import org.apache.commons.cli.CommandLine;
26  import org.apache.commons.cli.CommandLineParser;
27  import org.apache.commons.cli.HelpFormatter;
28  import org.apache.commons.cli.Options;
29  import org.apache.commons.cli.ParseException;
30  import org.apache.commons.cli.PosixParser;
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.fs.FileSystem;
35  import org.apache.hadoop.fs.Path;
36  import org.apache.hadoop.hbase.HBaseConfiguration;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.KeyValue;
39  import org.apache.hadoop.hbase.io.compress.Compression;
40  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
41  import org.apache.hadoop.hbase.io.hfile.HFile;
42  import org.apache.hadoop.hbase.io.hfile.HFileBlockIndex;
43  import org.apache.hadoop.hbase.util.BloomFilterFactory;
44  import org.apache.hadoop.io.BytesWritable;
45  
46  /**
47   * Creates an HFile with random key/value pairs.
48   */
49  public class CreateRandomStoreFile {
50  
51    /**
52     * As much as this number of bytes can be added or subtracted from key/value
53     * lengths.
54     */
55    private static final int LEN_VARIATION = 5;
56  
57    private static final Log LOG =
58        LogFactory.getLog(CreateRandomStoreFile.class);
59    private static final String OUTPUT_DIR_OPTION = "o";
60    private static final String NUM_KV_OPTION = "n";
61    private static final String HFILE_VERSION_OPTION = "h";
62    private static final String KEY_SIZE_OPTION = "k";
63    private static final String VALUE_SIZE_OPTION = "v";
64    private static final String COMPRESSION_OPTION = "c";
65    private static final String BLOOM_FILTER_OPTION = "bf";
66    private static final String BLOCK_SIZE_OPTION = "bs";
67    private static final String BLOOM_BLOCK_SIZE_OPTION = "bfbs";
68    private static final String INDEX_BLOCK_SIZE_OPTION = "ibs";
69  
70    /** The exit code this command-line tool returns on failure */
71    private static final int EXIT_FAILURE = 1;
72  
73    /** The number of valid key types in a store file */
74    private static final int NUM_VALID_KEY_TYPES =
75        KeyValue.Type.values().length - 2;
76  
77    private Options options = new Options();
78  
79    private int keyPrefixLen, keyLen, rowLen, cfLen, valueLen;
80    private Random rand;
81  
82    /**
83     * Runs the tools.
84     *
85     * @param args command-line arguments
86     * @return true in case of success
87     * @throws IOException
88     */
89    public boolean run(String[] args) throws IOException {
90      options.addOption(OUTPUT_DIR_OPTION, "output_dir", true,
91          "Output directory");
92      options.addOption(NUM_KV_OPTION, "num_kv", true,
93          "Number of key/value pairs");
94      options.addOption(KEY_SIZE_OPTION, "key_size", true, "Average key size");
95      options.addOption(VALUE_SIZE_OPTION, "value_size", true,
96          "Average value size");
97      options.addOption(HFILE_VERSION_OPTION, "hfile_version", true,
98          "HFile version to create");
99      options.addOption(COMPRESSION_OPTION, "compression", true,
100         " Compression type, one of "
101             + Arrays.toString(Compression.Algorithm.values()));
102     options.addOption(BLOOM_FILTER_OPTION, "bloom_filter", true,
103         "Bloom filter type, one of "
104             + Arrays.toString(BloomType.values()));
105     options.addOption(BLOCK_SIZE_OPTION, "block_size", true,
106         "HFile block size");
107     options.addOption(BLOOM_BLOCK_SIZE_OPTION, "bloom_block_size", true,
108         "Compound Bloom filters block size");
109     options.addOption(INDEX_BLOCK_SIZE_OPTION, "index_block_size", true,
110         "Index block size");
111 
112     if (args.length == 0) {
113       HelpFormatter formatter = new HelpFormatter();
114       formatter.printHelp(CreateRandomStoreFile.class.getSimpleName(), options,
115           true);
116       return false;
117     }
118 
119     CommandLineParser parser = new PosixParser();
120     CommandLine cmdLine;
121     try {
122       cmdLine = parser.parse(options, args);
123     } catch (ParseException ex) {
124       LOG.error(ex);
125       return false;
126     }
127 
128     if (!cmdLine.hasOption(OUTPUT_DIR_OPTION)) {
129       LOG.error("Output directory is not specified");
130       return false;
131     }
132 
133     if (!cmdLine.hasOption(NUM_KV_OPTION)) {
134       LOG.error("The number of keys/values not specified");
135       return false;
136     }
137 
138     if (!cmdLine.hasOption(KEY_SIZE_OPTION)) {
139       LOG.error("Key size is not specified");
140       return false;
141     }
142 
143     if (!cmdLine.hasOption(VALUE_SIZE_OPTION)) {
144       LOG.error("Value size not specified");
145       return false;
146     }
147 
148     Configuration conf = HBaseConfiguration.create();
149 
150     Path outputDir = new Path(cmdLine.getOptionValue(OUTPUT_DIR_OPTION));
151 
152     long numKV = Long.parseLong(cmdLine.getOptionValue(NUM_KV_OPTION));
153     configureKeyValue(numKV,
154         Integer.parseInt(cmdLine.getOptionValue(KEY_SIZE_OPTION)),
155         Integer.parseInt(cmdLine.getOptionValue(VALUE_SIZE_OPTION)));
156 
157     FileSystem fs = FileSystem.get(conf);
158 
159     Compression.Algorithm compr = Compression.Algorithm.NONE;
160     if (cmdLine.hasOption(COMPRESSION_OPTION)) {
161       compr = Compression.Algorithm.valueOf(
162           cmdLine.getOptionValue(COMPRESSION_OPTION));
163     }
164 
165     BloomType bloomType = BloomType.NONE;
166     if (cmdLine.hasOption(BLOOM_FILTER_OPTION)) {
167       bloomType = BloomType.valueOf(cmdLine.getOptionValue(
168           BLOOM_FILTER_OPTION));
169     }
170 
171     int blockSize = HConstants.DEFAULT_BLOCKSIZE;
172     if (cmdLine.hasOption(BLOCK_SIZE_OPTION))
173       blockSize = Integer.valueOf(cmdLine.getOptionValue(BLOCK_SIZE_OPTION));
174     
175     if (cmdLine.hasOption(BLOOM_BLOCK_SIZE_OPTION)) {
176       conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_BLOCK_SIZE,
177           Integer.valueOf(cmdLine.getOptionValue(BLOOM_BLOCK_SIZE_OPTION)));
178     }
179     
180     if (cmdLine.hasOption(INDEX_BLOCK_SIZE_OPTION)) {
181       conf.setInt(HFileBlockIndex.MAX_CHUNK_SIZE_KEY,
182           Integer.valueOf(cmdLine.getOptionValue(INDEX_BLOCK_SIZE_OPTION)));
183     }
184 
185     StoreFile.Writer sfw = new StoreFile.WriterBuilder(conf,
186         new CacheConfig(conf), fs, blockSize)
187             .withOutputDir(outputDir)
188             .withCompression(compr)
189             .withBloomType(bloomType)
190             .withMaxKeyCount(numKV)
191             .withChecksumType(HFile.DEFAULT_CHECKSUM_TYPE)
192             .withBytesPerChecksum(HFile.DEFAULT_BYTES_PER_CHECKSUM)
193             .build();
194 
195     rand = new Random();
196     LOG.info("Writing " + numKV + " key/value pairs");
197     for (long i = 0; i < numKV; ++i) {
198       sfw.append(generateKeyValue(i));
199     }
200 
201     int numMetaBlocks = rand.nextInt(10) + 1;
202     LOG.info("Writing " + numMetaBlocks + " meta blocks");
203     for (int metaI = 0; metaI < numMetaBlocks; ++metaI) {
204       sfw.getHFileWriter().appendMetaBlock(generateString(),
205           new BytesWritable(generateValue()));
206     }
207     sfw.close();
208 
209     Path storeFilePath = sfw.getPath();
210     long fileSize = fs.getFileStatus(storeFilePath).getLen();
211     LOG.info("Created " + storeFilePath + ", " + fileSize + " bytes");
212 
213     return true;
214   }
215 
216   private void configureKeyValue(long numKV, int keyLen, int valueLen) {
217     numKV = Math.abs(numKV);
218     keyLen = Math.abs(keyLen);
219     keyPrefixLen = 0;
220     while (numKV != 0) {
221       numKV >>>= 8;
222       ++keyPrefixLen;
223     }
224 
225     this.keyLen = Math.max(keyPrefixLen, keyLen);
226     this.valueLen = valueLen;
227 
228     // Arbitrarily split the key into row, column family, and qualifier.
229     rowLen = keyPrefixLen / 3;
230     cfLen = keyPrefixLen / 4;
231   }
232 
233   private int nextInRange(int range) {
234     return rand.nextInt(2 * range + 1) - range;
235   }
236 
237   public KeyValue generateKeyValue(long i) {
238     byte[] k = generateKey(i);
239     byte[] v = generateValue();
240 
241     return new KeyValue(
242         k, 0, rowLen,
243         k, rowLen, cfLen,
244         k, rowLen + cfLen, k.length - rowLen - cfLen,
245         rand.nextLong(),
246         generateKeyType(rand),
247         v, 0, v.length);
248   }
249 
250   public static KeyValue.Type generateKeyType(Random rand) {
251     if (rand.nextBoolean()) {
252       // Let's make half of KVs puts.
253       return KeyValue.Type.Put;
254     } else {
255       KeyValue.Type keyType =
256           KeyValue.Type.values()[1 + rand.nextInt(NUM_VALID_KEY_TYPES)];
257       if (keyType == KeyValue.Type.Minimum || keyType == KeyValue.Type.Maximum)
258       {
259         throw new RuntimeException("Generated an invalid key type: " + keyType
260             + ". " + "Probably the layout of KeyValue.Type has changed.");
261       }
262       return keyType;
263     }
264   }
265 
266   private String generateString() {
267     StringBuilder sb = new StringBuilder();
268     for (int i = 0; i < rand.nextInt(10); ++i) {
269       sb.append((char) ('A' + rand.nextInt(26)));
270     }
271     return sb.toString();
272   }
273 
274   private byte[] generateKey(long i) {
275     byte[] k = new byte[Math.max(keyPrefixLen, keyLen
276         + nextInRange(LEN_VARIATION))];
277     for (int pos = keyPrefixLen - 1; pos >= 0; --pos) {
278       k[pos] = (byte) (i & 0xFF);
279       i >>>= 8;
280     }
281     for (int pos = keyPrefixLen; pos < k.length; ++pos) {
282       k[pos] = (byte) rand.nextInt(256);
283     }
284     return k;
285   }
286 
287   private byte[] generateValue() {
288     byte[] v = new byte[Math.max(1, valueLen + nextInRange(LEN_VARIATION))];
289     for (int i = 0; i < v.length; ++i) {
290       v[i] = (byte) rand.nextInt(256);
291     }
292     return v;
293   }
294 
295   public static void main(String[] args) {
296     CreateRandomStoreFile app = new CreateRandomStoreFile();
297     try {
298       if (!app.run(args))
299         System.exit(EXIT_FAILURE);
300     } catch (IOException ex) {
301       LOG.error(ex);
302       System.exit(EXIT_FAILURE);
303     }
304 
305   }
306 
307 }