1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.IOException;
23 import java.nio.ByteBuffer;
24 import java.util.ArrayList;
25 import java.util.Arrays;
26 import java.util.Collections;
27 import java.util.Comparator;
28 import java.util.List;
29 import java.util.TreeSet;
30
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.HBaseTestCase;
37 import org.apache.hadoop.hbase.HBaseTestingUtility;
38 import org.apache.hadoop.hbase.HConstants;
39 import org.apache.hadoop.hbase.KeyValue;
40 import org.apache.hadoop.hbase.client.Scan;
41 import org.apache.hadoop.hbase.io.Reference.Range;
42 import org.apache.hadoop.hbase.io.hfile.BlockCache;
43 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
44 import org.apache.hadoop.hbase.io.hfile.CacheStats;
45 import org.apache.hadoop.hbase.io.hfile.HFile;
46 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
47 import org.apache.hadoop.hbase.util.BloomFilterFactory;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.hdfs.MiniDFSCluster;
50 import org.mockito.Mockito;
51
52 import com.google.common.base.Joiner;
53 import com.google.common.collect.Iterables;
54 import com.google.common.collect.Lists;
55
56
57
58
59 public class TestStoreFile extends HBaseTestCase {
60 static final Log LOG = LogFactory.getLog(TestStoreFile.class);
61 private CacheConfig cacheConf = new CacheConfig(conf);
62 private String ROOT_DIR;
63
64 @Override
65 public void setUp() throws Exception {
66 super.setUp();
67 ROOT_DIR = new Path(this.testDir, "TestStoreFile").toString();
68 }
69
70
71
72
73
74
75 public void testBasicHalfMapFile() throws Exception {
76
77 StoreFile.Writer writer = StoreFile.createWriter(this.fs,
78 new Path(new Path(this.testDir, "regionname"), "familyname"), 2 * 1024,
79 conf, cacheConf);
80 writeStoreFile(writer);
81 checkHalfHFile(new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
82 StoreFile.BloomType.NONE));
83 }
84
85 private void writeStoreFile(final StoreFile.Writer writer) throws IOException {
86 writeStoreFile(writer, Bytes.toBytes(getName()), Bytes.toBytes(getName()));
87 }
88
89
90
91
92
93
94 public static void writeStoreFile(final StoreFile.Writer writer, byte[] fam, byte[] qualifier)
95 throws IOException {
96 long now = System.currentTimeMillis();
97 try {
98 for (char d = FIRST_CHAR; d <= LAST_CHAR; d++) {
99 for (char e = FIRST_CHAR; e <= LAST_CHAR; e++) {
100 byte[] b = new byte[] { (byte) d, (byte) e };
101 writer.append(new KeyValue(b, fam, qualifier, now, b));
102 }
103 }
104 } finally {
105 writer.close();
106 }
107 }
108
109
110
111
112
113
114 public void testReference()
115 throws IOException {
116 Path storedir = new Path(new Path(this.testDir, "regionname"), "familyname");
117 Path dir = new Path(storedir, "1234567890");
118
119 StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
120 conf, cacheConf);
121 writeStoreFile(writer);
122 StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
123 StoreFile.BloomType.NONE);
124 StoreFile.Reader reader = hsf.createReader();
125
126
127
128 KeyValue kv = KeyValue.createKeyValueFromKey(reader.midkey());
129 byte [] midRow = kv.getRow();
130 kv = KeyValue.createKeyValueFromKey(reader.getLastKey());
131 byte [] finalRow = kv.getRow();
132
133 Path refPath = StoreFile.split(fs, dir, hsf, midRow, Range.top);
134 StoreFile refHsf = new StoreFile(this.fs, refPath, conf, cacheConf,
135 StoreFile.BloomType.NONE);
136
137
138 HFileScanner s = refHsf.createReader().getScanner(false, false);
139 for(boolean first = true; (!s.isSeeked() && s.seekTo()) || s.next();) {
140 ByteBuffer bb = s.getKey();
141 kv = KeyValue.createKeyValueFromKey(bb);
142 if (first) {
143 assertTrue(Bytes.equals(kv.getRow(), midRow));
144 first = false;
145 }
146 }
147 assertTrue(Bytes.equals(kv.getRow(), finalRow));
148 }
149
150 private void checkHalfHFile(final StoreFile f)
151 throws IOException {
152 byte [] midkey = f.createReader().midkey();
153 KeyValue midKV = KeyValue.createKeyValueFromKey(midkey);
154 byte [] midRow = midKV.getRow();
155
156 Path topDir = Store.getStoreHomedir(this.testDir, "1",
157 Bytes.toBytes(f.getPath().getParent().getName()));
158 if (this.fs.exists(topDir)) {
159 this.fs.delete(topDir, true);
160 }
161 Path topPath = StoreFile.split(this.fs, topDir, f, midRow, Range.top);
162
163 Path bottomDir = Store.getStoreHomedir(this.testDir, "2",
164 Bytes.toBytes(f.getPath().getParent().getName()));
165 if (this.fs.exists(bottomDir)) {
166 this.fs.delete(bottomDir, true);
167 }
168 Path bottomPath = StoreFile.split(this.fs, bottomDir,
169 f, midRow, Range.bottom);
170
171 StoreFile.Reader top = new StoreFile(this.fs, topPath, conf, cacheConf,
172 StoreFile.BloomType.NONE).createReader();
173 StoreFile.Reader bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
174 StoreFile.BloomType.NONE).createReader();
175 ByteBuffer previous = null;
176 LOG.info("Midkey: " + midKV.toString());
177 ByteBuffer bbMidkeyBytes = ByteBuffer.wrap(midkey);
178 try {
179
180
181
182
183 boolean first = true;
184 ByteBuffer key = null;
185 HFileScanner topScanner = top.getScanner(false, false);
186 while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
187 (topScanner.isSeeked() && topScanner.next())) {
188 key = topScanner.getKey();
189
190 if (topScanner.getReader().getComparator().compare(key.array(),
191 key.arrayOffset(), key.limit(), midkey, 0, midkey.length) < 0) {
192 fail("key=" + Bytes.toStringBinary(key) + " < midkey=" +
193 Bytes.toStringBinary(midkey));
194 }
195 if (first) {
196 first = false;
197 LOG.info("First in top: " + Bytes.toString(Bytes.toBytes(key)));
198 }
199 }
200 LOG.info("Last in top: " + Bytes.toString(Bytes.toBytes(key)));
201
202 first = true;
203 HFileScanner bottomScanner = bottom.getScanner(false, false);
204 while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
205 bottomScanner.next()) {
206 previous = bottomScanner.getKey();
207 key = bottomScanner.getKey();
208 if (first) {
209 first = false;
210 LOG.info("First in bottom: " +
211 Bytes.toString(Bytes.toBytes(previous)));
212 }
213 assertTrue(key.compareTo(bbMidkeyBytes) < 0);
214 }
215 if (previous != null) {
216 LOG.info("Last in bottom: " + Bytes.toString(Bytes.toBytes(previous)));
217 }
218
219 this.fs.delete(topPath, false);
220 this.fs.delete(bottomPath, false);
221
222
223
224
225 byte [] badmidkey = Bytes.toBytes(" .");
226 topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
227 bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
228 Range.bottom);
229 top = new StoreFile(this.fs, topPath, conf, cacheConf,
230 StoreFile.BloomType.NONE).createReader();
231 bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
232 StoreFile.BloomType.NONE).createReader();
233 bottomScanner = bottom.getScanner(false, false);
234 int count = 0;
235 while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
236 bottomScanner.next()) {
237 count++;
238 }
239
240 assertTrue(count == 0);
241
242 first = true;
243 topScanner = top.getScanner(false, false);
244 while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
245 topScanner.next()) {
246 key = topScanner.getKey();
247 assertTrue(topScanner.getReader().getComparator().compare(key.array(),
248 key.arrayOffset(), key.limit(), badmidkey, 0, badmidkey.length) >= 0);
249 if (first) {
250 first = false;
251 KeyValue keyKV = KeyValue.createKeyValueFromKey(key);
252 LOG.info("First top when key < bottom: " + keyKV);
253 String tmp = Bytes.toString(keyKV.getRow());
254 for (int i = 0; i < tmp.length(); i++) {
255 assertTrue(tmp.charAt(i) == 'a');
256 }
257 }
258 }
259 KeyValue keyKV = KeyValue.createKeyValueFromKey(key);
260 LOG.info("Last top when key < bottom: " + keyKV);
261 String tmp = Bytes.toString(keyKV.getRow());
262 for (int i = 0; i < tmp.length(); i++) {
263 assertTrue(tmp.charAt(i) == 'z');
264 }
265
266 this.fs.delete(topPath, false);
267 this.fs.delete(bottomPath, false);
268
269
270 badmidkey = Bytes.toBytes("|||");
271 topPath = StoreFile.split(this.fs, topDir, f, badmidkey, Range.top);
272 bottomPath = StoreFile.split(this.fs, bottomDir, f, badmidkey,
273 Range.bottom);
274 top = new StoreFile(this.fs, topPath, conf, cacheConf,
275 StoreFile.BloomType.NONE).createReader();
276 bottom = new StoreFile(this.fs, bottomPath, conf, cacheConf,
277 StoreFile.BloomType.NONE).createReader();
278 first = true;
279 bottomScanner = bottom.getScanner(false, false);
280 while ((!bottomScanner.isSeeked() && bottomScanner.seekTo()) ||
281 bottomScanner.next()) {
282 key = bottomScanner.getKey();
283 if (first) {
284 first = false;
285 keyKV = KeyValue.createKeyValueFromKey(key);
286 LOG.info("First bottom when key > top: " + keyKV);
287 tmp = Bytes.toString(keyKV.getRow());
288 for (int i = 0; i < tmp.length(); i++) {
289 assertTrue(tmp.charAt(i) == 'a');
290 }
291 }
292 }
293 keyKV = KeyValue.createKeyValueFromKey(key);
294 LOG.info("Last bottom when key > top: " + keyKV);
295 for (int i = 0; i < tmp.length(); i++) {
296 assertTrue(Bytes.toString(keyKV.getRow()).charAt(i) == 'z');
297 }
298 count = 0;
299 topScanner = top.getScanner(false, false);
300 while ((!topScanner.isSeeked() && topScanner.seekTo()) ||
301 (topScanner.isSeeked() && topScanner.next())) {
302 count++;
303 }
304
305 assertTrue(count == 0);
306 } finally {
307 if (top != null) {
308 top.close(true);
309 }
310 if (bottom != null) {
311 bottom.close(true);
312 }
313 fs.delete(f.getPath(), true);
314 }
315 }
316
317 private static final String localFormatter = "%010d";
318
319 private void bloomWriteRead(StoreFile.Writer writer, FileSystem fs)
320 throws Exception {
321 float err = conf.getFloat(
322 BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, 0);
323 Path f = writer.getPath();
324 long now = System.currentTimeMillis();
325 for (int i = 0; i < 2000; i += 2) {
326 String row = String.format(localFormatter, i);
327 KeyValue kv = new KeyValue(row.getBytes(), "family".getBytes(),
328 "col".getBytes(), now, "value".getBytes());
329 writer.append(kv);
330 }
331 writer.close();
332
333 StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
334 reader.loadFileInfo();
335 reader.loadBloomfilter();
336 StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
337
338
339 int falsePos = 0;
340 int falseNeg = 0;
341 for (int i = 0; i < 2000; i++) {
342 String row = String.format(localFormatter, i);
343 TreeSet<byte[]> columns = new TreeSet<byte[]>();
344 columns.add("family:col".getBytes());
345
346 Scan scan = new Scan(row.getBytes(),row.getBytes());
347 scan.addColumn("family".getBytes(), "family:col".getBytes());
348 boolean exists = scanner.shouldSeek(scan, columns);
349 if (i % 2 == 0) {
350 if (!exists) falseNeg++;
351 } else {
352 if (exists) falsePos++;
353 }
354 }
355 reader.close(true);
356 fs.delete(f, true);
357 assertEquals("False negatives: " + falseNeg, 0, falseNeg);
358 int maxFalsePos = (int) (2 * 2000 * err);
359 assertTrue("Too many false positives: " + falsePos + " (err=" + err
360 + ", expected no more than " + maxFalsePos + ")",
361 falsePos <= maxFalsePos);
362 }
363
364 public void testBloomFilter() throws Exception {
365 FileSystem fs = FileSystem.getLocal(conf);
366 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE,
367 (float) 0.01);
368 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
369
370
371 Path f = new Path(ROOT_DIR, getName());
372 StoreFile.Writer writer = new StoreFile.Writer(fs, f,
373 StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
374 conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
375
376 bloomWriteRead(writer, fs);
377 }
378
379 public void testBloomTypes() throws Exception {
380 float err = (float) 0.01;
381 FileSystem fs = FileSystem.getLocal(conf);
382 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
383 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
384
385 int rowCount = 50;
386 int colCount = 10;
387 int versions = 2;
388
389
390 StoreFile.BloomType[] bt =
391 {StoreFile.BloomType.ROWCOL, StoreFile.BloomType.ROW};
392 int[] expKeys = {rowCount*colCount, rowCount};
393
394
395
396
397 float[] expErr = {2*rowCount*colCount*err, 2*rowCount*2*colCount*err};
398
399 for (int x : new int[]{0,1}) {
400
401 Path f = new Path(ROOT_DIR, getName() + x);
402 StoreFile.Writer writer = new StoreFile.Writer(fs, f,
403 StoreFile.DEFAULT_BLOCKSIZE_SMALL,
404 HFile.DEFAULT_COMPRESSION_ALGORITHM,
405 conf, cacheConf, KeyValue.COMPARATOR, bt[x], expKeys[x]);
406
407 long now = System.currentTimeMillis();
408 for (int i = 0; i < rowCount*2; i += 2) {
409 for (int j = 0; j < colCount*2; j += 2) {
410 String row = String.format(localFormatter, i);
411 String col = String.format(localFormatter, j);
412 for (int k= 0; k < versions; ++k) {
413 KeyValue kv = new KeyValue(row.getBytes(),
414 "family".getBytes(), ("col" + col).getBytes(),
415 now-k, Bytes.toBytes((long)-1));
416 writer.append(kv);
417 }
418 }
419 }
420 writer.close();
421
422 StoreFile.Reader reader = new StoreFile.Reader(fs, f, cacheConf);
423 reader.loadFileInfo();
424 reader.loadBloomfilter();
425 StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
426 assertEquals(expKeys[x], reader.bloomFilter.getKeyCount());
427
428
429 int falsePos = 0;
430 int falseNeg = 0;
431 for (int i = 0; i < rowCount*2; ++i) {
432 for (int j = 0; j < colCount*2; ++j) {
433 String row = String.format(localFormatter, i);
434 String col = String.format(localFormatter, j);
435 TreeSet<byte[]> columns = new TreeSet<byte[]>();
436 columns.add(("col" + col).getBytes());
437
438 Scan scan = new Scan(row.getBytes(),row.getBytes());
439 scan.addColumn("family".getBytes(), ("col"+col).getBytes());
440 boolean exists = scanner.shouldSeek(scan, columns);
441 boolean shouldRowExist = i % 2 == 0;
442 boolean shouldColExist = j % 2 == 0;
443 shouldColExist = shouldColExist || bt[x] == StoreFile.BloomType.ROW;
444 if (shouldRowExist && shouldColExist) {
445 if (!exists) falseNeg++;
446 } else {
447 if (exists) falsePos++;
448 }
449 }
450 }
451 reader.close(true);
452 fs.delete(f, true);
453 System.out.println(bt[x].toString());
454 System.out.println(" False negatives: " + falseNeg);
455 System.out.println(" False positives: " + falsePos);
456 assertEquals(0, falseNeg);
457 assertTrue(falsePos < 2*expErr[x]);
458 }
459 }
460
461 public void testBloomEdgeCases() throws Exception {
462 float err = (float)0.005;
463 FileSystem fs = FileSystem.getLocal(conf);
464 Path f = new Path(ROOT_DIR, getName());
465 conf.setFloat(BloomFilterFactory.IO_STOREFILE_BLOOM_ERROR_RATE, err);
466 conf.setBoolean(BloomFilterFactory.IO_STOREFILE_BLOOM_ENABLED, true);
467 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS, 1000);
468
469
470 conf.setInt(HFile.FORMAT_VERSION_KEY, 1);
471
472
473 StoreFile.Writer writer = new StoreFile.Writer(fs, f,
474 StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
475 conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW, 2000);
476 assertFalse(writer.hasBloom());
477 writer.close();
478 fs.delete(f, true);
479
480 conf.setInt(BloomFilterFactory.IO_STOREFILE_BLOOM_MAX_KEYS,
481 Integer.MAX_VALUE);
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496 writer = new StoreFile.Writer(fs, f,
497 StoreFile.DEFAULT_BLOCKSIZE_SMALL, HFile.DEFAULT_COMPRESSION_ALGORITHM,
498 conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.ROW,
499 Integer.MAX_VALUE);
500 assertFalse(writer.hasBloom());
501 writer.close();
502 fs.delete(f, true);
503 }
504
505 public void testFlushTimeComparator() {
506 assertOrdering(StoreFile.Comparators.FLUSH_TIME,
507 mockStoreFile(true, 1000, -1, "/foo/123"),
508 mockStoreFile(true, 1000, -1, "/foo/126"),
509 mockStoreFile(true, 2000, -1, "/foo/126"),
510 mockStoreFile(false, -1, 1, "/foo/1"),
511 mockStoreFile(false, -1, 3, "/foo/2"),
512 mockStoreFile(false, -1, 5, "/foo/2"),
513 mockStoreFile(false, -1, 5, "/foo/3"));
514 }
515
516
517
518
519
520 private void assertOrdering(Comparator<StoreFile> comparator, StoreFile ... sfs) {
521 ArrayList<StoreFile> sorted = Lists.newArrayList(sfs);
522 Collections.shuffle(sorted);
523 Collections.sort(sorted, comparator);
524 LOG.debug("sfs: " + Joiner.on(",").join(sfs));
525 LOG.debug("sorted: " + Joiner.on(",").join(sorted));
526 assertTrue(Iterables.elementsEqual(Arrays.asList(sfs), sorted));
527 }
528
529
530
531
532 private StoreFile mockStoreFile(boolean bulkLoad, long bulkTimestamp,
533 long seqId, String path) {
534 StoreFile mock = Mockito.mock(StoreFile.class);
535 Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult();
536 Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
537 if (bulkLoad) {
538
539 Mockito.doThrow(new IllegalAccessError("bulk load"))
540 .when(mock).getMaxSequenceId();
541 } else {
542 Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
543 }
544 Mockito.doReturn(new Path(path)).when(mock).getPath();
545 String name = "mock storefile, bulkLoad=" + bulkLoad +
546 " bulkTimestamp=" + bulkTimestamp +
547 " seqId=" + seqId +
548 " path=" + path;
549 Mockito.doReturn(name).when(mock).toString();
550 return mock;
551 }
552
553
554
555
556
557
558
559
560
561 List<KeyValue> getKeyValueSet(long[] timestamps, int numRows,
562 byte[] qualifier, byte[] family) {
563 List<KeyValue> kvList = new ArrayList<KeyValue>();
564 for (int i=1;i<=numRows;i++) {
565 byte[] b = Bytes.toBytes(i) ;
566 LOG.info(Bytes.toString(b));
567 LOG.info(Bytes.toString(b));
568 for (long timestamp: timestamps)
569 {
570 kvList.add(new KeyValue(b, family, qualifier, timestamp, b));
571 }
572 }
573 return kvList;
574 }
575
576
577
578
579
580 public void testMultipleTimestamps() throws IOException {
581 byte[] family = Bytes.toBytes("familyname");
582 byte[] qualifier = Bytes.toBytes("qualifier");
583 int numRows = 10;
584 long[] timestamps = new long[] {20,10,5,1};
585 Scan scan = new Scan();
586
587 Path storedir = new Path(new Path(this.testDir, "regionname"),
588 "familyname");
589 Path dir = new Path(storedir, "1234567890");
590 StoreFile.Writer writer = StoreFile.createWriter(this.fs, dir, 8 * 1024,
591 conf, cacheConf);
592
593 List<KeyValue> kvList = getKeyValueSet(timestamps,numRows,
594 family, qualifier);
595
596 for (KeyValue kv : kvList) {
597 writer.append(kv);
598 }
599 writer.appendMetadata(0, false);
600 writer.close();
601
602 StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
603 StoreFile.BloomType.NONE);
604 StoreFile.Reader reader = hsf.createReader();
605 StoreFileScanner scanner = reader.getStoreFileScanner(false, false);
606 TreeSet<byte[]> columns = new TreeSet<byte[]>();
607 columns.add(qualifier);
608
609 scan.setTimeRange(20, 100);
610 assertTrue(scanner.shouldSeek(scan, columns));
611
612 scan.setTimeRange(1, 2);
613 assertTrue(scanner.shouldSeek(scan, columns));
614
615 scan.setTimeRange(8, 10);
616 assertTrue(scanner.shouldSeek(scan, columns));
617
618 scan.setTimeRange(7, 50);
619 assertTrue(scanner.shouldSeek(scan, columns));
620
621
622
623
624
625 }
626
627 public void testCacheOnWriteEvictOnClose() throws Exception {
628 Configuration conf = this.conf;
629
630
631 Path baseDir = new Path(new Path(this.testDir, "regionname"),"twoCOWEOC");
632
633
634 BlockCache bc = new CacheConfig(conf).getBlockCache();
635 assertNotNull(bc);
636 CacheStats cs = bc.getStats();
637 long startHit = cs.getHitCount();
638 long startMiss = cs.getMissCount();
639 long startEvicted = cs.getEvictedCount();
640
641
642 conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, false);
643 CacheConfig cacheConf = new CacheConfig(conf);
644 Path pathCowOff = new Path(baseDir, "123456789");
645 StoreFile.Writer writer = writeStoreFile(conf, cacheConf, pathCowOff, 3);
646 StoreFile hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
647 StoreFile.BloomType.NONE);
648 LOG.debug(hsf.getPath().toString());
649
650
651 StoreFile.Reader reader = hsf.createReader();
652 reader.loadFileInfo();
653 StoreFileScanner scanner = reader.getStoreFileScanner(true, true);
654 scanner.seek(KeyValue.LOWESTKEY);
655 while (scanner.next() != null);
656 assertEquals(startHit, cs.getHitCount());
657 assertEquals(startMiss + 3, cs.getMissCount());
658 assertEquals(startEvicted, cs.getEvictedCount());
659 startMiss += 3;
660 scanner.close();
661 reader.close(cacheConf.shouldEvictOnClose());
662
663
664 conf.setBoolean(CacheConfig.CACHE_BLOCKS_ON_WRITE_KEY, true);
665 cacheConf = new CacheConfig(conf);
666 Path pathCowOn = new Path(baseDir, "123456788");
667 writer = writeStoreFile(conf, cacheConf, pathCowOn, 3);
668 hsf = new StoreFile(this.fs, writer.getPath(), conf, cacheConf,
669 StoreFile.BloomType.NONE);
670
671
672 reader = hsf.createReader();
673 scanner = reader.getStoreFileScanner(true, true);
674 scanner.seek(KeyValue.LOWESTKEY);
675 while (scanner.next() != null);
676 assertEquals(startHit + 3, cs.getHitCount());
677 assertEquals(startMiss, cs.getMissCount());
678 assertEquals(startEvicted, cs.getEvictedCount());
679 startHit += 3;
680 scanner.close();
681 reader.close(cacheConf.shouldEvictOnClose());
682
683
684 hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
685 StoreFile.BloomType.NONE);
686 StoreFile.Reader readerOne = hsf.createReader();
687 readerOne.loadFileInfo();
688 StoreFileScanner scannerOne = readerOne.getStoreFileScanner(true, true);
689 scannerOne.seek(KeyValue.LOWESTKEY);
690 hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
691 StoreFile.BloomType.NONE);
692 StoreFile.Reader readerTwo = hsf.createReader();
693 readerTwo.loadFileInfo();
694 StoreFileScanner scannerTwo = readerTwo.getStoreFileScanner(true, true);
695 scannerTwo.seek(KeyValue.LOWESTKEY);
696 KeyValue kv1 = null;
697 KeyValue kv2 = null;
698 while ((kv1 = scannerOne.next()) != null) {
699 kv2 = scannerTwo.next();
700 assertTrue(kv1.equals(kv2));
701 assertTrue(Bytes.compareTo(
702 kv1.getBuffer(), kv1.getKeyOffset(), kv1.getKeyLength(),
703 kv2.getBuffer(), kv2.getKeyOffset(), kv2.getKeyLength()) == 0);
704 assertTrue(Bytes.compareTo(
705 kv1.getBuffer(), kv1.getValueOffset(), kv1.getValueLength(),
706 kv2.getBuffer(), kv2.getValueOffset(), kv2.getValueLength()) == 0);
707 }
708 assertNull(scannerTwo.next());
709 assertEquals(startHit + 6, cs.getHitCount());
710 assertEquals(startMiss, cs.getMissCount());
711 assertEquals(startEvicted, cs.getEvictedCount());
712 startHit += 6;
713 scannerOne.close();
714 readerOne.close(cacheConf.shouldEvictOnClose());
715 scannerTwo.close();
716 readerTwo.close(cacheConf.shouldEvictOnClose());
717
718
719 conf.setBoolean("hbase.rs.evictblocksonclose", true);
720 cacheConf = new CacheConfig(conf);
721 hsf = new StoreFile(this.fs, pathCowOff, conf, cacheConf,
722 StoreFile.BloomType.NONE);
723 reader = hsf.createReader();
724 reader.close(cacheConf.shouldEvictOnClose());
725
726
727 assertEquals(startHit, cs.getHitCount());
728 assertEquals(startMiss, cs.getMissCount());
729 assertEquals(startEvicted + 3, cs.getEvictedCount());
730 startEvicted += 3;
731
732
733 conf.setBoolean("hbase.rs.evictblocksonclose", false);
734 cacheConf = new CacheConfig(conf);
735 hsf = new StoreFile(this.fs, pathCowOn, conf, cacheConf,
736 StoreFile.BloomType.NONE);
737 reader = hsf.createReader();
738 reader.close(cacheConf.shouldEvictOnClose());
739
740
741 assertEquals(startHit, cs.getHitCount());
742 assertEquals(startMiss, cs.getMissCount());
743 assertEquals(startEvicted, cs.getEvictedCount());
744 }
745
746 private StoreFile.Writer writeStoreFile(Configuration conf,
747 CacheConfig cacheConf, Path path, int numBlocks)
748 throws IOException {
749
750 int numKVs = 5 * numBlocks;
751 List<KeyValue> kvs = new ArrayList<KeyValue>(numKVs);
752 byte [] b = Bytes.toBytes("x");
753 int totalSize = 0;
754 for (int i=numKVs;i>0;i--) {
755 KeyValue kv = new KeyValue(b, b, b, i, b);
756 kvs.add(kv);
757
758 totalSize += kv.getLength() + 1;
759 }
760 int blockSize = totalSize / numBlocks;
761 StoreFile.Writer writer = new StoreFile.Writer(fs, path, blockSize,
762 HFile.DEFAULT_COMPRESSION_ALGORITHM,
763 conf, cacheConf, KeyValue.COMPARATOR, StoreFile.BloomType.NONE, 2000);
764
765 kvs.remove(kvs.size()-1);
766 for (KeyValue kv : kvs) {
767 writer.append(kv);
768 }
769 writer.appendMetadata(0, false);
770 writer.close();
771 return writer;
772 }
773 }