1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase;
20
21 import java.io.IOException;
22 import java.nio.ByteBuffer;
23 import java.security.SecureRandom;
24 import java.util.Random;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.commons.math.random.RandomData;
29 import org.apache.commons.math.random.RandomDataImpl;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.fs.FileSystem;
32 import org.apache.hadoop.fs.Path;
33 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
34 import org.apache.hadoop.hbase.io.crypto.Encryption;
35 import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting;
36 import org.apache.hadoop.hbase.io.crypto.aes.AES;
37 import org.apache.hadoop.hbase.io.hfile.AbstractHFileWriter;
38 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
39 import org.apache.hadoop.hbase.io.hfile.HFile;
40 import org.apache.hadoop.hbase.io.hfile.HFileContext;
41 import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
42 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
43 import org.apache.hadoop.hbase.util.Bytes;
44
45
46
47
48
49
50 public class HFilePerformanceEvaluation {
51
52 private static final int ROW_LENGTH = 10;
53 private static final int ROW_COUNT = 1000000;
54 private static final int RFILE_BLOCKSIZE = 8 * 1024;
55 private static StringBuilder testSummary = new StringBuilder();
56
57
58 static {
59 System.setProperty("org.apache.commons.logging.Log",
60 "org.apache.commons.logging.impl.SimpleLog");
61 System.setProperty("org.apache.commons.logging.simplelog.log.org.apache.hadoop.io.compress.CodecPool",
62 "WARN");
63 }
64
65 static final Log LOG =
66 LogFactory.getLog(HFilePerformanceEvaluation.class.getName());
67
68 static byte [] format(final int i) {
69 String v = Integer.toString(i);
70 return Bytes.toBytes("0000000000".substring(v.length()) + v);
71 }
72
73 static ImmutableBytesWritable format(final int i, ImmutableBytesWritable w) {
74 w.set(format(i));
75 return w;
76 }
77
78
79
80
81
82
83 private void runBenchmarks() throws Exception {
84 final Configuration conf = new Configuration();
85 final FileSystem fs = FileSystem.get(conf);
86 final Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
87
88
89 runWriteBenchmark(conf, fs, mf, "none", "none");
90 runReadBenchmark(conf, fs, mf, "none", "none");
91
92
93 runWriteBenchmark(conf, fs, mf, "gz", "none");
94 runReadBenchmark(conf, fs, mf, "gz", "none");
95
96
97 final Configuration aesconf = new Configuration();
98 aesconf.set(HConstants.CRYPTO_KEYPROVIDER_CONF_KEY, KeyProviderForTesting.class.getName());
99 aesconf.set(HConstants.CRYPTO_MASTERKEY_NAME_CONF_KEY, "hbase");
100 aesconf.setInt("hfile.format.version", 3);
101 final FileSystem aesfs = FileSystem.get(aesconf);
102 final Path aesmf = aesfs.makeQualified(new Path("performanceevaluation.aes.mapfile"));
103
104
105 runWriteBenchmark(aesconf, aesfs, aesmf, "none", "aes");
106 runReadBenchmark(aesconf, aesfs, aesmf, "none", "aes");
107
108
109 runWriteBenchmark(aesconf, aesfs, aesmf, "gz", "aes");
110 runReadBenchmark(aesconf, aesfs, aesmf, "gz", "aes");
111
112
113 if (fs.exists(mf)) {
114 fs.delete(mf, true);
115 }
116 if (aesfs.exists(aesmf)) {
117 aesfs.delete(aesmf, true);
118 }
119
120
121 LOG.info("\n***************\n" + "Result Summary" + "\n***************\n");
122 LOG.info(testSummary.toString());
123
124 }
125
126
127
128
129
130
131
132
133
134
135 private void runWriteBenchmark(Configuration conf, FileSystem fs, Path mf, String codec,
136 String cipher) throws Exception {
137 if (fs.exists(mf)) {
138 fs.delete(mf, true);
139 }
140
141 runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT, codec, cipher),
142 ROW_COUNT, codec, cipher);
143
144 }
145
146
147
148
149
150
151
152
153
154 private void runReadBenchmark(final Configuration conf, final FileSystem fs, final Path mf,
155 final String codec, final String cipher) {
156 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
157 public void run() {
158 try {
159 runBenchmark(new UniformRandomSmallScan(conf, fs, mf, ROW_COUNT),
160 ROW_COUNT, codec, cipher);
161 } catch (Exception e) {
162 testSummary.append("UniformRandomSmallScan failed " + e.getMessage());
163 e.printStackTrace();
164 }
165 }
166 });
167
168 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
169 public void run() {
170 try {
171 runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
172 ROW_COUNT, codec, cipher);
173 } catch (Exception e) {
174 testSummary.append("UniformRandomReadBenchmark failed " + e.getMessage());
175 e.printStackTrace();
176 }
177 }
178 });
179
180 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
181 public void run() {
182 try {
183 runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
184 ROW_COUNT, codec, cipher);
185 } catch (Exception e) {
186 testSummary.append("GaussianRandomReadBenchmark failed " + e.getMessage());
187 e.printStackTrace();
188 }
189 }
190 });
191
192 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
193 public void run() {
194 try {
195 runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
196 ROW_COUNT, codec, cipher);
197 } catch (Exception e) {
198 testSummary.append("SequentialReadBenchmark failed " + e.getMessage());
199 e.printStackTrace();
200 }
201 }
202 });
203
204 }
205
206 protected void runBenchmark(RowOrientedBenchmark benchmark, int rowCount,
207 String codec, String cipher) throws Exception {
208 LOG.info("Running " + benchmark.getClass().getSimpleName() + " with codec[" +
209 codec + "] " + "cipher[" + cipher + "] for " + rowCount + " rows.");
210
211 long elapsedTime = benchmark.run();
212
213 LOG.info("Running " + benchmark.getClass().getSimpleName() + " with codec[" +
214 codec + "] " + "cipher[" + cipher + "] for " + rowCount + " rows took " +
215 elapsedTime + "ms.");
216
217
218 testSummary.append("Running ").append(benchmark.getClass().getSimpleName())
219 .append(" with codec[").append(codec).append("] cipher[").append(cipher)
220 .append("] for ").append(rowCount).append(" rows took ").append(elapsedTime)
221 .append("ms.").append("\n");
222 }
223
224 static abstract class RowOrientedBenchmark {
225
226 protected final Configuration conf;
227 protected final FileSystem fs;
228 protected final Path mf;
229 protected final int totalRows;
230 protected String codec = "none";
231 protected String cipher = "none";
232
233 public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
234 int totalRows, String codec, String cipher) {
235 this.conf = conf;
236 this.fs = fs;
237 this.mf = mf;
238 this.totalRows = totalRows;
239 this.codec = codec;
240 this.cipher = cipher;
241 }
242
243 public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
244 int totalRows) {
245 this.conf = conf;
246 this.fs = fs;
247 this.mf = mf;
248 this.totalRows = totalRows;
249 }
250
251 void setUp() throws Exception {
252
253 }
254
255 abstract void doRow(int i) throws Exception;
256
257 protected int getReportingPeriod() {
258 return this.totalRows / 10;
259 }
260
261 void tearDown() throws Exception {
262
263 }
264
265
266
267
268
269
270 long run() throws Exception {
271 long elapsedTime;
272 setUp();
273 long startTime = System.currentTimeMillis();
274 try {
275 for (int i = 0; i < totalRows; i++) {
276 if (i > 0 && i % getReportingPeriod() == 0) {
277 LOG.info("Processed " + i + " rows.");
278 }
279 doRow(i);
280 }
281 elapsedTime = System.currentTimeMillis() - startTime;
282 } finally {
283 tearDown();
284 }
285 return elapsedTime;
286 }
287
288 }
289
290 static class SequentialWriteBenchmark extends RowOrientedBenchmark {
291 protected HFile.Writer writer;
292 private Random random = new Random();
293 private byte[] bytes = new byte[ROW_LENGTH];
294
295 public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf,
296 int totalRows, String codec, String cipher) {
297 super(conf, fs, mf, totalRows, codec, cipher);
298 }
299
300 @Override
301 void setUp() throws Exception {
302
303 HFileContextBuilder builder = new HFileContextBuilder()
304 .withCompression(AbstractHFileWriter.compressionByName(codec))
305 .withBlockSize(RFILE_BLOCKSIZE);
306
307 if (cipher == "aes") {
308 byte[] cipherKey = new byte[AES.KEY_LENGTH];
309 new SecureRandom().nextBytes(cipherKey);
310 builder.withEncryptionContext(Encryption.newContext(conf)
311 .setCipher(Encryption.getCipher(conf, cipher))
312 .setKey(cipherKey));
313 } else if (!"none".equals(cipher)) {
314 throw new IOException("Cipher " + cipher + " not supported.");
315 }
316
317 HFileContext hFileContext = builder.build();
318
319 writer = HFile.getWriterFactoryNoCache(conf)
320 .withPath(fs, mf)
321 .withFileContext(hFileContext)
322 .withComparator(new KeyValue.RawBytesComparator())
323 .create();
324 }
325
326 @Override
327 void doRow(int i) throws Exception {
328 writer.append(format(i), generateValue());
329 }
330
331 private byte[] generateValue() {
332 random.nextBytes(bytes);
333 return bytes;
334 }
335
336 @Override
337 protected int getReportingPeriod() {
338 return this.totalRows;
339 }
340
341 @Override
342 void tearDown() throws Exception {
343 writer.close();
344 }
345
346 }
347
348 static abstract class ReadBenchmark extends RowOrientedBenchmark {
349
350 protected HFile.Reader reader;
351
352 public ReadBenchmark(Configuration conf, FileSystem fs, Path mf,
353 int totalRows) {
354 super(conf, fs, mf, totalRows);
355 }
356
357 @Override
358 void setUp() throws Exception {
359 reader = HFile.createReader(this.fs, this.mf, new CacheConfig(this.conf), this.conf);
360 this.reader.loadFileInfo();
361 }
362
363 @Override
364 void tearDown() throws Exception {
365 reader.close();
366 }
367
368 }
369
370 static class SequentialReadBenchmark extends ReadBenchmark {
371 private HFileScanner scanner;
372
373 public SequentialReadBenchmark(Configuration conf, FileSystem fs,
374 Path mf, int totalRows) {
375 super(conf, fs, mf, totalRows);
376 }
377
378 @Override
379 void setUp() throws Exception {
380 super.setUp();
381 this.scanner = this.reader.getScanner(false, false);
382 this.scanner.seekTo();
383 }
384
385 @Override
386 void doRow(int i) throws Exception {
387 if (this.scanner.next()) {
388 ByteBuffer k = this.scanner.getKey();
389 PerformanceEvaluationCommons.assertKey(format(i + 1), k);
390 ByteBuffer v = scanner.getValue();
391 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
392 }
393 }
394
395 @Override
396 protected int getReportingPeriod() {
397 return this.totalRows;
398 }
399
400 }
401
402 static class UniformRandomReadBenchmark extends ReadBenchmark {
403
404 private Random random = new Random();
405
406 public UniformRandomReadBenchmark(Configuration conf, FileSystem fs,
407 Path mf, int totalRows) {
408 super(conf, fs, mf, totalRows);
409 }
410
411 @Override
412 void doRow(int i) throws Exception {
413 HFileScanner scanner = this.reader.getScanner(false, true);
414 byte [] b = getRandomRow();
415 if (scanner.seekTo(b) < 0) {
416 LOG.info("Not able to seekTo " + new String(b));
417 return;
418 }
419 ByteBuffer k = scanner.getKey();
420 PerformanceEvaluationCommons.assertKey(b, k);
421 ByteBuffer v = scanner.getValue();
422 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
423 }
424
425 private byte [] getRandomRow() {
426 return format(random.nextInt(totalRows));
427 }
428 }
429
430 static class UniformRandomSmallScan extends ReadBenchmark {
431 private Random random = new Random();
432
433 public UniformRandomSmallScan(Configuration conf, FileSystem fs,
434 Path mf, int totalRows) {
435 super(conf, fs, mf, totalRows/10);
436 }
437
438 @Override
439 void doRow(int i) throws Exception {
440 HFileScanner scanner = this.reader.getScanner(false, false);
441 byte [] b = getRandomRow();
442 if (scanner.seekTo(b) != 0) {
443 LOG.info("Nonexistent row: " + new String(b));
444 return;
445 }
446 ByteBuffer k = scanner.getKey();
447 PerformanceEvaluationCommons.assertKey(b, k);
448
449 for (int ii = 0; ii < 30; ii++) {
450 if (!scanner.next()) {
451 LOG.info("NOTHING FOLLOWS");
452 return;
453 }
454 ByteBuffer v = scanner.getValue();
455 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
456 }
457 }
458
459 private byte [] getRandomRow() {
460 return format(random.nextInt(totalRows));
461 }
462 }
463
464 static class GaussianRandomReadBenchmark extends ReadBenchmark {
465
466 private RandomData randomData = new RandomDataImpl();
467
468 public GaussianRandomReadBenchmark(Configuration conf, FileSystem fs,
469 Path mf, int totalRows) {
470 super(conf, fs, mf, totalRows);
471 }
472
473 @Override
474 void doRow(int i) throws Exception {
475 HFileScanner scanner = this.reader.getScanner(false, true);
476 byte[] gaussianRandomRowBytes = getGaussianRandomRowBytes();
477 scanner.seekTo(gaussianRandomRowBytes);
478 for (int ii = 0; ii < 30; ii++) {
479 if (!scanner.next()) {
480 LOG.info("NOTHING FOLLOWS");
481 return;
482 }
483 scanner.getKey();
484 scanner.getValue();
485 }
486 }
487
488 private byte [] getGaussianRandomRowBytes() {
489 int r = (int) randomData.nextGaussian((double)totalRows / 2.0,
490 (double)totalRows / 10.0);
491
492 return format(Math.min(totalRows, Math.max(r,0)));
493 }
494 }
495
496
497
498
499
500
501 public static void main(String[] args) throws Exception {
502 new HFilePerformanceEvaluation().runBenchmarks();
503 }
504 }