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.util.Random;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.commons.math.random.RandomData;
28 import org.apache.commons.math.random.RandomDataImpl;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.fs.FileSystem;
31 import org.apache.hadoop.fs.Path;
32 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33 import org.apache.hadoop.hbase.io.compress.Compression;
34 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
35 import org.apache.hadoop.hbase.io.hfile.HFile;
36 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
37 import org.apache.hadoop.hbase.util.Bytes;
38
39
40
41
42
43
44 public class HFilePerformanceEvaluation {
45
46 private static final int ROW_LENGTH = 10;
47 private static final int ROW_COUNT = 1000000;
48 private static final int RFILE_BLOCKSIZE = 8 * 1024;
49
50 static final Log LOG =
51 LogFactory.getLog(HFilePerformanceEvaluation.class.getName());
52
53 static byte [] format(final int i) {
54 String v = Integer.toString(i);
55 return Bytes.toBytes("0000000000".substring(v.length()) + v);
56 }
57
58 static ImmutableBytesWritable format(final int i, ImmutableBytesWritable w) {
59 w.set(format(i));
60 return w;
61 }
62
63 private void runBenchmarks() throws Exception {
64 final Configuration conf = new Configuration();
65 final FileSystem fs = FileSystem.get(conf);
66 final Path mf = fs.makeQualified(new Path("performanceevaluation.mapfile"));
67 if (fs.exists(mf)) {
68 fs.delete(mf, true);
69 }
70
71 runBenchmark(new SequentialWriteBenchmark(conf, fs, mf, ROW_COUNT),
72 ROW_COUNT);
73 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
74 public void run() {
75 try {
76 runBenchmark(new UniformRandomSmallScan(conf, fs, mf, ROW_COUNT),
77 ROW_COUNT);
78 } catch (Exception e) {
79 e.printStackTrace();
80 }
81 }
82 });
83 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
84 public void run() {
85 try {
86 runBenchmark(new UniformRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
87 ROW_COUNT);
88 } catch (Exception e) {
89 e.printStackTrace();
90 }
91 }
92 });
93 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
94 public void run() {
95 try {
96 runBenchmark(new GaussianRandomReadBenchmark(conf, fs, mf, ROW_COUNT),
97 ROW_COUNT);
98 } catch (Exception e) {
99 e.printStackTrace();
100 }
101 }
102 });
103 PerformanceEvaluationCommons.concurrentReads(new Runnable() {
104 public void run() {
105 try {
106 runBenchmark(new SequentialReadBenchmark(conf, fs, mf, ROW_COUNT),
107 ROW_COUNT);
108 } catch (Exception e) {
109 e.printStackTrace();
110 }
111 }
112 });
113
114 }
115
116 protected void runBenchmark(RowOrientedBenchmark benchmark, int rowCount)
117 throws Exception {
118 LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
119 rowCount + " rows.");
120 long elapsedTime = benchmark.run();
121 LOG.info("Running " + benchmark.getClass().getSimpleName() + " for " +
122 rowCount + " rows took " + elapsedTime + "ms.");
123 }
124
125 static abstract class RowOrientedBenchmark {
126
127 protected final Configuration conf;
128 protected final FileSystem fs;
129 protected final Path mf;
130 protected final int totalRows;
131
132 public RowOrientedBenchmark(Configuration conf, FileSystem fs, Path mf,
133 int totalRows) {
134 this.conf = conf;
135 this.fs = fs;
136 this.mf = mf;
137 this.totalRows = totalRows;
138 }
139
140 void setUp() throws Exception {
141
142 }
143
144 abstract void doRow(int i) throws Exception;
145
146 protected int getReportingPeriod() {
147 return this.totalRows / 10;
148 }
149
150 void tearDown() throws Exception {
151
152 }
153
154
155
156
157
158
159 long run() throws Exception {
160 long elapsedTime;
161 setUp();
162 long startTime = System.currentTimeMillis();
163 try {
164 for (int i = 0; i < totalRows; i++) {
165 if (i > 0 && i % getReportingPeriod() == 0) {
166 LOG.info("Processed " + i + " rows.");
167 }
168 doRow(i);
169 }
170 elapsedTime = System.currentTimeMillis() - startTime;
171 } finally {
172 tearDown();
173 }
174 return elapsedTime;
175 }
176
177 }
178
179 static class SequentialWriteBenchmark extends RowOrientedBenchmark {
180 protected HFile.Writer writer;
181 private Random random = new Random();
182 private byte[] bytes = new byte[ROW_LENGTH];
183
184 public SequentialWriteBenchmark(Configuration conf, FileSystem fs, Path mf,
185 int totalRows) {
186 super(conf, fs, mf, totalRows);
187 }
188
189 @Override
190 void setUp() throws Exception {
191 writer =
192 HFile.getWriterFactoryNoCache(conf)
193 .withPath(fs, mf)
194 .withBlockSize(RFILE_BLOCKSIZE)
195 .withComparator(new KeyValue.RawBytesComparator())
196 .create();
197 }
198
199 @Override
200 void doRow(int i) throws Exception {
201 writer.append(format(i), generateValue());
202 }
203
204 private byte[] generateValue() {
205 random.nextBytes(bytes);
206 return bytes;
207 }
208
209 @Override
210 protected int getReportingPeriod() {
211 return this.totalRows;
212 }
213
214 @Override
215 void tearDown() throws Exception {
216 writer.close();
217 }
218
219 }
220
221 static abstract class ReadBenchmark extends RowOrientedBenchmark {
222
223 protected HFile.Reader reader;
224
225 public ReadBenchmark(Configuration conf, FileSystem fs, Path mf,
226 int totalRows) {
227 super(conf, fs, mf, totalRows);
228 }
229
230 @Override
231 void setUp() throws Exception {
232 reader = HFile.createReader(this.fs, this.mf, new CacheConfig(this.conf));
233 this.reader.loadFileInfo();
234 }
235
236 @Override
237 void tearDown() throws Exception {
238 reader.close();
239 }
240
241 }
242
243 static class SequentialReadBenchmark extends ReadBenchmark {
244 private HFileScanner scanner;
245
246 public SequentialReadBenchmark(Configuration conf, FileSystem fs,
247 Path mf, int totalRows) {
248 super(conf, fs, mf, totalRows);
249 }
250
251 @Override
252 void setUp() throws Exception {
253 super.setUp();
254 this.scanner = this.reader.getScanner(false, false);
255 this.scanner.seekTo();
256 }
257
258 @Override
259 void doRow(int i) throws Exception {
260 if (this.scanner.next()) {
261 ByteBuffer k = this.scanner.getKey();
262 PerformanceEvaluationCommons.assertKey(format(i + 1), k);
263 ByteBuffer v = scanner.getValue();
264 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
265 }
266 }
267
268 @Override
269 protected int getReportingPeriod() {
270 return this.totalRows;
271 }
272
273 }
274
275 static class UniformRandomReadBenchmark extends ReadBenchmark {
276
277 private Random random = new Random();
278
279 public UniformRandomReadBenchmark(Configuration conf, FileSystem fs,
280 Path mf, int totalRows) {
281 super(conf, fs, mf, totalRows);
282 }
283
284 @Override
285 void doRow(int i) throws Exception {
286 HFileScanner scanner = this.reader.getScanner(false, true);
287 byte [] b = getRandomRow();
288 if (scanner.seekTo(b) < 0) {
289 LOG.info("Not able to seekTo " + new String(b));
290 return;
291 }
292 ByteBuffer k = scanner.getKey();
293 PerformanceEvaluationCommons.assertKey(b, k);
294 ByteBuffer v = scanner.getValue();
295 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
296 }
297
298 private byte [] getRandomRow() {
299 return format(random.nextInt(totalRows));
300 }
301 }
302
303 static class UniformRandomSmallScan extends ReadBenchmark {
304 private Random random = new Random();
305
306 public UniformRandomSmallScan(Configuration conf, FileSystem fs,
307 Path mf, int totalRows) {
308 super(conf, fs, mf, totalRows/10);
309 }
310
311 @Override
312 void doRow(int i) throws Exception {
313 HFileScanner scanner = this.reader.getScanner(false, false);
314 byte [] b = getRandomRow();
315 if (scanner.seekTo(b) != 0) {
316 LOG.info("Nonexistent row: " + new String(b));
317 return;
318 }
319 ByteBuffer k = scanner.getKey();
320 PerformanceEvaluationCommons.assertKey(b, k);
321
322 for (int ii = 0; ii < 30; ii++) {
323 if (!scanner.next()) {
324 LOG.info("NOTHING FOLLOWS");
325 return;
326 }
327 ByteBuffer v = scanner.getValue();
328 PerformanceEvaluationCommons.assertValueSize(v.limit(), ROW_LENGTH);
329 }
330 }
331
332 private byte [] getRandomRow() {
333 return format(random.nextInt(totalRows));
334 }
335 }
336
337 static class GaussianRandomReadBenchmark extends ReadBenchmark {
338
339 private RandomData randomData = new RandomDataImpl();
340
341 public GaussianRandomReadBenchmark(Configuration conf, FileSystem fs,
342 Path mf, int totalRows) {
343 super(conf, fs, mf, totalRows);
344 }
345
346 @Override
347 void doRow(int i) throws Exception {
348 HFileScanner scanner = this.reader.getScanner(false, true);
349 byte[] gaussianRandomRowBytes = getGaussianRandomRowBytes();
350 if (scanner.seekTo(gaussianRandomRowBytes) < 0) {
351 LOG.info("Not able to seekTo " + new String(gaussianRandomRowBytes));
352 return;
353 }
354 for (int ii = 0; ii < 30; ii++) {
355 if (!scanner.next()) {
356 LOG.info("NOTHING FOLLOWS");
357 return;
358 }
359 scanner.getKey();
360 scanner.getValue();
361 }
362 }
363
364 private byte [] getGaussianRandomRowBytes() {
365 int r = (int) randomData.nextGaussian((double)totalRows / 2.0,
366 (double)totalRows / 10.0);
367 return format(r);
368 }
369 }
370
371
372
373
374
375
376 public static void main(String[] args) throws Exception {
377 new HFilePerformanceEvaluation().runBenchmarks();
378 }
379 }