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 org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.fs.FileSystem;
26 import org.apache.hadoop.fs.Path;
27 import org.apache.hadoop.hbase.KeyValue;
28 import org.apache.hadoop.hbase.KeyValue.KVComparator;
29 import org.apache.hadoop.hbase.client.Scan;
30 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
31 import org.apache.hadoop.hbase.io.Reference;
32 import org.apache.hadoop.hbase.io.hfile.BlockCache;
33 import org.apache.hadoop.hbase.io.hfile.Compression;
34 import org.apache.hadoop.hbase.io.hfile.HFile;
35 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
36 import org.apache.hadoop.hbase.io.hfile.LruBlockCache;
37 import org.apache.hadoop.hbase.util.BloomFilter;
38 import org.apache.hadoop.hbase.util.ByteBloomFilter;
39 import org.apache.hadoop.hbase.util.Bytes;
40 import org.apache.hadoop.hbase.util.Hash;
41 import org.apache.hadoop.hbase.util.Writables;
42 import org.apache.hadoop.io.RawComparator;
43 import org.apache.hadoop.io.WritableUtils;
44 import org.apache.hadoop.util.StringUtils;
45
46 import com.google.common.base.Function;
47 import com.google.common.collect.ImmutableList;
48 import com.google.common.collect.Ordering;
49
50 import java.io.FileNotFoundException;
51 import java.io.IOException;
52 import java.lang.management.ManagementFactory;
53 import java.lang.management.MemoryUsage;
54 import java.nio.ByteBuffer;
55 import java.text.NumberFormat;
56 import java.util.Arrays;
57 import java.util.Collections;
58 import java.util.Comparator;
59 import java.util.List;
60 import java.util.Map;
61 import java.util.Random;
62 import java.util.SortedSet;
63 import java.util.concurrent.atomic.AtomicBoolean;
64 import java.util.regex.Matcher;
65 import java.util.regex.Pattern;
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80 public class StoreFile {
81 static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
82
83
84 static final String IO_STOREFILE_BLOOM_ERROR_RATE = "io.storefile.bloom.error.rate";
85 static final String IO_STOREFILE_BLOOM_MAX_FOLD = "io.storefile.bloom.max.fold";
86 static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
87 static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
88
89 public static enum BloomType {
90
91
92
93 NONE,
94
95
96
97 ROW,
98
99
100
101 ROWCOL
102 }
103
104
105 public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
106
107 public static final byte [] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
108
109 static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
110
111 static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
112
113
114 static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
115
116 static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
117
118
119
120 public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
121
122
123 private static BlockCache hfileBlockCache = null;
124
125 private final FileSystem fs;
126
127 private final Path path;
128
129 private Reference reference;
130
131 private Path referencePath;
132
133 private boolean blockcache;
134
135 private boolean inMemory;
136
137
138
139 private long sequenceid = -1;
140
141
142
143 private AtomicBoolean majorCompaction = null;
144
145
146 public static final byte[] BULKLOAD_TASK_KEY =
147 Bytes.toBytes("BULKLOAD_SOURCE_TASK");
148 public static final byte[] BULKLOAD_TIME_KEY =
149 Bytes.toBytes("BULKLOAD_TIMESTAMP");
150
151
152
153
154 private Map<byte[], byte[]> metadataMap;
155
156
157
158
159
160
161 private static final Pattern REF_NAME_PARSER =
162 Pattern.compile("^(\\d+)(?:\\.(.+))?$");
163
164
165 private volatile Reader reader;
166
167
168 private final static Random rand = new Random();
169 private final Configuration conf;
170 private final BloomType bloomType;
171
172
173
174
175
176
177
178
179
180
181
182
183
184 StoreFile(final FileSystem fs,
185 final Path p,
186 final boolean blockcache,
187 final Configuration conf,
188 final BloomType bt,
189 final boolean inMemory)
190 throws IOException {
191 this.conf = conf;
192 this.fs = fs;
193 this.path = p;
194 this.blockcache = blockcache;
195 this.inMemory = inMemory;
196 if (isReference(p)) {
197 this.reference = Reference.read(fs, p);
198 this.referencePath = getReferredToFile(this.path);
199 }
200
201
202 if (conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
203 this.bloomType = bt;
204 } else {
205 this.bloomType = BloomType.NONE;
206 LOG.info("Ignoring bloom filter check for file (disabled in config)");
207 }
208 }
209
210
211
212
213 Path getPath() {
214 return this.path;
215 }
216
217
218
219
220 byte [] getFamily() {
221 return Bytes.toBytes(this.path.getParent().getName());
222 }
223
224
225
226
227
228 boolean isReference() {
229 return this.reference != null;
230 }
231
232
233
234
235
236 public static boolean isReference(final Path p) {
237 return !p.getName().startsWith("_") &&
238 isReference(p, REF_NAME_PARSER.matcher(p.getName()));
239 }
240
241
242
243
244
245
246 public static boolean isReference(final Path p, final Matcher m) {
247 if (m == null || !m.matches()) {
248 LOG.warn("Failed match of store file name " + p.toString());
249 throw new RuntimeException("Failed match of store file name " +
250 p.toString());
251 }
252 return m.groupCount() > 1 && m.group(2) != null;
253 }
254
255
256
257
258
259
260
261
262 static Path getReferredToFile(final Path p) {
263 Matcher m = REF_NAME_PARSER.matcher(p.getName());
264 if (m == null || !m.matches()) {
265 LOG.warn("Failed match of store file name " + p.toString());
266 throw new RuntimeException("Failed match of store file name " +
267 p.toString());
268 }
269
270 String otherRegion = m.group(2);
271
272 Path tableDir = p.getParent().getParent().getParent();
273 String nameStrippedOfSuffix = m.group(1);
274
275
276 return new Path(new Path(new Path(tableDir, otherRegion),
277 p.getParent().getName()), nameStrippedOfSuffix);
278 }
279
280
281
282
283 boolean isMajorCompaction() {
284 if (this.majorCompaction == null) {
285 throw new NullPointerException("This has not been set yet");
286 }
287 return this.majorCompaction.get();
288 }
289
290
291
292
293 public long getMaxSequenceId() {
294 if (this.sequenceid == -1) {
295 throw new IllegalAccessError("Has not been initialized");
296 }
297 return this.sequenceid;
298 }
299
300
301
302
303
304
305
306
307
308 public static long getMaxSequenceIdInList(List<StoreFile> sfs) {
309 long max = 0;
310 for (StoreFile sf : sfs) {
311 if (!sf.isBulkLoadResult()) {
312 max = Math.max(max, sf.getMaxSequenceId());
313 }
314 }
315 return max;
316 }
317
318
319
320
321
322 boolean isBulkLoadResult() {
323 return metadataMap.containsKey(BULKLOAD_TIME_KEY);
324 }
325
326
327
328
329 public long getBulkLoadTimestamp() {
330 return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
331 }
332
333
334
335
336
337
338
339 public static synchronized BlockCache getBlockCache(Configuration conf) {
340 if (hfileBlockCache != null) return hfileBlockCache;
341
342 float cachePercentage = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.0f);
343
344 if (cachePercentage == 0L) return null;
345 if (cachePercentage > 1.0) {
346 throw new IllegalArgumentException(HFILE_BLOCK_CACHE_SIZE_KEY +
347 " must be between 0.0 and 1.0, not > 1.0");
348 }
349
350
351 MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
352 long cacheSize = (long)(mu.getMax() * cachePercentage);
353 LOG.info("Allocating LruBlockCache with maximum size " +
354 StringUtils.humanReadableInt(cacheSize));
355 hfileBlockCache = new LruBlockCache(cacheSize, DEFAULT_BLOCKSIZE_SMALL);
356 return hfileBlockCache;
357 }
358
359
360
361
362 public BlockCache getBlockCache() {
363 return blockcache ? getBlockCache(conf) : null;
364 }
365
366
367
368
369
370
371
372 private Reader open() throws IOException {
373
374 if (this.reader != null) {
375 throw new IllegalAccessError("Already open");
376 }
377
378 if (isReference()) {
379 this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
380 getBlockCache(), this.reference);
381 } else {
382 this.reader = new Reader(this.fs, this.path, getBlockCache(),
383 this.inMemory);
384 }
385
386
387 metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
388
389 byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
390 if (b != null) {
391
392
393
394
395
396 this.sequenceid = Bytes.toLong(b);
397 if (isReference()) {
398 if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
399 this.sequenceid += 1;
400 }
401 }
402 }
403
404 b = metadataMap.get(MAJOR_COMPACTION_KEY);
405 if (b != null) {
406 boolean mc = Bytes.toBoolean(b);
407 if (this.majorCompaction == null) {
408 this.majorCompaction = new AtomicBoolean(mc);
409 } else {
410 this.majorCompaction.set(mc);
411 }
412 }
413
414 if (this.bloomType != BloomType.NONE) {
415 this.reader.loadBloomfilter();
416 }
417
418 try {
419 byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
420 if (timerangeBytes != null) {
421 this.reader.timeRangeTracker = new TimeRangeTracker();
422 Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
423 }
424 } catch (IllegalArgumentException e) {
425 LOG.error("Error reading timestamp range data from meta -- " +
426 "proceeding without", e);
427 this.reader.timeRangeTracker = null;
428 }
429 return this.reader;
430 }
431
432
433
434
435
436 public Reader createReader() throws IOException {
437 if (this.reader == null) {
438 this.reader = open();
439 }
440 return this.reader;
441 }
442
443
444
445
446
447
448 public Reader getReader() {
449 return this.reader;
450 }
451
452
453
454
455 public synchronized void closeReader() throws IOException {
456 if (this.reader != null) {
457 this.reader.close();
458 this.reader = null;
459 }
460 }
461
462
463
464
465
466 public void deleteReader() throws IOException {
467 closeReader();
468 this.fs.delete(getPath(), true);
469 }
470
471 @Override
472 public String toString() {
473 return this.path.toString() +
474 (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
475 }
476
477
478
479
480 public String toStringDetailed() {
481 StringBuilder sb = new StringBuilder();
482 sb.append(this.path.toString());
483 sb.append(", isReference=").append(isReference());
484 sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
485 if (isBulkLoadResult()) {
486 sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
487 } else {
488 sb.append(", seqid=").append(getMaxSequenceId());
489 }
490 sb.append(", majorCompaction=").append(isMajorCompaction());
491
492 return sb.toString();
493 }
494
495
496
497
498
499
500
501
502
503 public static Path rename(final FileSystem fs,
504 final Path src,
505 final Path tgt)
506 throws IOException {
507
508 if (!fs.exists(src)) {
509 throw new FileNotFoundException(src.toString());
510 }
511 if (!fs.rename(src, tgt)) {
512 throw new IOException("Failed rename of " + src + " to " + tgt);
513 }
514 return tgt;
515 }
516
517
518
519
520
521
522
523
524
525
526
527 public static Writer createWriter(final FileSystem fs,
528 final Path dir,
529 final int blocksize)
530 throws IOException {
531
532 return createWriter(fs, dir, blocksize, null, null, null, BloomType.NONE, 0);
533 }
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550 public static StoreFile.Writer createWriter(final FileSystem fs,
551 final Path dir,
552 final int blocksize,
553 final Compression.Algorithm algorithm,
554 final KeyValue.KVComparator c,
555 final Configuration conf,
556 BloomType bloomType,
557 int maxKeySize)
558 throws IOException {
559
560 if (!fs.exists(dir)) {
561 fs.mkdirs(dir);
562 }
563 Path path = getUniqueFile(fs, dir);
564 if(conf == null || !conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
565 bloomType = BloomType.NONE;
566 }
567
568 return new Writer(fs, path, blocksize,
569 algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
570 conf, c == null? KeyValue.COMPARATOR: c, bloomType, maxKeySize);
571 }
572
573
574
575
576
577
578 public static Path getUniqueFile(final FileSystem fs, final Path dir)
579 throws IOException {
580 if (!fs.getFileStatus(dir).isDir()) {
581 throw new IOException("Expecting " + dir.toString() +
582 " to be a directory");
583 }
584 return fs.getFileStatus(dir).isDir()? getRandomFilename(fs, dir): dir;
585 }
586
587
588
589
590
591
592
593
594 static Path getRandomFilename(final FileSystem fs, final Path dir)
595 throws IOException {
596 return getRandomFilename(fs, dir, null);
597 }
598
599
600
601
602
603
604
605
606
607 static Path getRandomFilename(final FileSystem fs,
608 final Path dir,
609 final String suffix)
610 throws IOException {
611 long id = -1;
612 Path p = null;
613 do {
614 id = Math.abs(rand.nextLong());
615 p = new Path(dir, Long.toString(id) +
616 ((suffix == null || suffix.length() <= 0)? "": suffix));
617 } while(fs.exists(p));
618 return p;
619 }
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635 static Path split(final FileSystem fs,
636 final Path splitDir,
637 final StoreFile f,
638 final byte [] splitRow,
639 final Reference.Range range)
640 throws IOException {
641
642 Reference r = new Reference(splitRow, range);
643
644
645
646
647 String parentRegionName = f.getPath().getParent().getParent().getName();
648
649
650 Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
651 return r.write(fs, p);
652 }
653
654
655
656
657
658
659 public static class Writer {
660 private final BloomFilter bloomFilter;
661 private final BloomType bloomType;
662 private KVComparator kvComparator;
663 private KeyValue lastKv = null;
664 private byte[] lastByteArray = null;
665 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
666
667
668
669
670
671
672 boolean isTimeRangeTrackerSet = false;
673
674 protected HFile.Writer writer;
675
676
677
678
679
680
681
682
683
684
685
686
687 public Writer(FileSystem fs, Path path, int blocksize,
688 Compression.Algorithm compress, final Configuration conf,
689 final KVComparator comparator, BloomType bloomType, int maxKeys)
690 throws IOException {
691 writer = new HFile.Writer(fs, path, blocksize, compress, comparator.getRawComparator());
692
693 this.kvComparator = comparator;
694
695 if (bloomType != BloomType.NONE && conf != null) {
696 float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
697
698
699
700
701 if (bloomType == BloomType.ROWCOL) {
702 err /= 2;
703 }
704 int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
705
706 this.bloomFilter = new ByteBloomFilter(maxKeys, err,
707 Hash.getHashType(conf), maxFold);
708 this.bloomFilter.allocBloom();
709 this.bloomType = bloomType;
710 } else {
711 this.bloomFilter = null;
712 this.bloomType = BloomType.NONE;
713 }
714 }
715
716
717
718
719
720
721
722
723 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
724 throws IOException {
725 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
726 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
727 Bytes.toBytes(majorCompaction));
728 appendTimeRangeMetadata();
729 }
730
731
732
733
734 public void appendTimeRangeMetadata() throws IOException {
735 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
736 }
737
738
739
740
741
742 public void setTimeRangeTracker(final TimeRangeTracker trt) {
743 this.timeRangeTracker = trt;
744 isTimeRangeTrackerSet = true;
745 }
746
747
748
749
750
751
752
753 public void includeInTimeRangeTracker(final KeyValue kv) {
754 if (!isTimeRangeTrackerSet) {
755 timeRangeTracker.includeTimestamp(kv);
756 }
757 }
758
759
760
761
762
763
764
765 public void includeInTimeRangeTracker(final byte [] key) {
766 if (!isTimeRangeTrackerSet) {
767 timeRangeTracker.includeTimestamp(key);
768 }
769 }
770
771 public void append(final KeyValue kv) throws IOException {
772 if (this.bloomFilter != null) {
773
774 boolean newKey = true;
775 if (this.lastKv != null) {
776 switch(bloomType) {
777 case ROW:
778 newKey = ! kvComparator.matchingRows(kv, lastKv);
779 break;
780 case ROWCOL:
781 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
782 break;
783 case NONE:
784 newKey = false;
785 }
786 }
787 if (newKey) {
788
789
790
791
792
793
794
795
796 switch (bloomType) {
797 case ROW:
798 this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
799 kv.getRowLength());
800 break;
801 case ROWCOL:
802
803 int ro = kv.getRowOffset();
804 int rl = kv.getRowLength();
805 int qo = kv.getQualifierOffset();
806 int ql = kv.getQualifierLength();
807 byte [] result = new byte[rl + ql];
808 System.arraycopy(kv.getBuffer(), ro, result, 0, rl);
809 System.arraycopy(kv.getBuffer(), qo, result, rl, ql);
810 this.bloomFilter.add(result);
811 break;
812 default:
813 }
814 this.lastKv = kv;
815 }
816 }
817 writer.append(kv);
818 includeInTimeRangeTracker(kv);
819 }
820
821 public Path getPath() {
822 return this.writer.getPath();
823 }
824
825 public void append(final byte [] key, final byte [] value) throws IOException {
826 if (this.bloomFilter != null) {
827
828 if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) {
829 this.bloomFilter.add(key);
830 this.lastByteArray = key;
831 }
832 }
833 writer.append(key, value);
834 includeInTimeRangeTracker(key);
835 }
836
837 public void close() throws IOException {
838
839 if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
840 bloomFilter.compactBloom();
841 if (this.bloomFilter.getMaxKeys() > 0) {
842 int b = this.bloomFilter.getByteSize();
843 int k = this.bloomFilter.getKeyCount();
844 int m = this.bloomFilter.getMaxKeys();
845 StoreFile.LOG.info("Bloom added to HFile. " + b + "B, " +
846 k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
847 ((double)k) / ((double)m)) + ")");
848 }
849 writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter());
850 writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter());
851 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
852 }
853 writer.close();
854 }
855
856 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
857 writer.appendFileInfo(key, value);
858 }
859 }
860
861
862
863
864 public static class Reader {
865 static final Log LOG = LogFactory.getLog(Reader.class.getName());
866
867 protected BloomFilter bloomFilter = null;
868 protected BloomType bloomFilterType;
869 private final HFile.Reader reader;
870 protected TimeRangeTracker timeRangeTracker = null;
871
872 public Reader(FileSystem fs, Path path, BlockCache blockCache, boolean inMemory)
873 throws IOException {
874 reader = new HFile.Reader(fs, path, blockCache, inMemory);
875 bloomFilterType = BloomType.NONE;
876 }
877
878 public RawComparator<byte []> getComparator() {
879 return reader.getComparator();
880 }
881
882
883
884
885
886
887
888
889 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
890 return new StoreFileScanner(this, getScanner(cacheBlocks, pread));
891 }
892
893
894
895
896
897
898
899
900
901
902 @Deprecated
903 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
904 return reader.getScanner(cacheBlocks, pread);
905 }
906
907 public void close() throws IOException {
908 reader.close();
909 }
910
911 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
912 return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns));
913 }
914
915
916
917
918
919
920 private boolean passesTimerangeFilter(Scan scan) {
921 if (timeRangeTracker == null) {
922 return true;
923 } else {
924 return timeRangeTracker.includesTimeRange(scan.getTimeRange());
925 }
926 }
927
928 private boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
929 if (this.bloomFilter == null || !scan.isGetScan()) {
930 return true;
931 }
932 byte[] row = scan.getStartRow();
933 byte[] key;
934 switch (this.bloomFilterType) {
935 case ROW:
936 key = row;
937 break;
938 case ROWCOL:
939 if (columns != null && columns.size() == 1) {
940 byte[] col = columns.first();
941 key = Bytes.add(row, col);
942 break;
943 }
944
945 default:
946 return true;
947 }
948
949 try {
950 ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
951 if (bloom != null) {
952 if (this.bloomFilterType == BloomType.ROWCOL) {
953
954
955
956
957 return this.bloomFilter.contains(key, bloom) ||
958 this.bloomFilter.contains(row, bloom);
959 }
960 else {
961 return this.bloomFilter.contains(key, bloom);
962 }
963 }
964 } catch (IOException e) {
965 LOG.error("Error reading bloom filter data -- proceeding without",
966 e);
967 setBloomFilterFaulty();
968 } catch (IllegalArgumentException e) {
969 LOG.error("Bad bloom filter data -- proceeding without", e);
970 setBloomFilterFaulty();
971 }
972
973 return true;
974 }
975
976 public Map<byte[], byte[]> loadFileInfo() throws IOException {
977 Map<byte [], byte []> fi = reader.loadFileInfo();
978
979 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
980 if (b != null) {
981 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
982 }
983
984 return fi;
985 }
986
987 public void loadBloomfilter() {
988 if (this.bloomFilter != null) {
989 return;
990 }
991
992 try {
993 ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false);
994 if (b != null) {
995 if (bloomFilterType == BloomType.NONE) {
996 throw new IOException("valid bloom filter type not found in FileInfo");
997 }
998
999
1000 this.bloomFilter = new ByteBloomFilter(b);
1001 LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col")
1002 + " bloom filter metadata for " + reader.getName());
1003 }
1004 } catch (IOException e) {
1005 LOG.error("Error reading bloom filter meta -- proceeding without", e);
1006 this.bloomFilter = null;
1007 } catch (IllegalArgumentException e) {
1008 LOG.error("Bad bloom filter meta -- proceeding without", e);
1009 this.bloomFilter = null;
1010 }
1011 }
1012
1013 public int getFilterEntries() {
1014 return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
1015 : reader.getFilterEntries();
1016 }
1017
1018 public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException {
1019 return reader.getMetaBlock(bloomFilterDataKey, cacheBlock);
1020 }
1021
1022 public void setBloomFilterFaulty() {
1023 bloomFilter = null;
1024 }
1025
1026 public byte[] getLastKey() {
1027 return reader.getLastKey();
1028 }
1029
1030 public byte[] midkey() throws IOException {
1031 return reader.midkey();
1032 }
1033
1034 public long length() {
1035 return reader.length();
1036 }
1037
1038 public int getEntries() {
1039 return reader.getEntries();
1040 }
1041
1042 public byte[] getFirstKey() {
1043 return reader.getFirstKey();
1044 }
1045
1046 public long indexSize() {
1047 return reader.indexSize();
1048 }
1049
1050 public BloomType getBloomFilterType() {
1051 return this.bloomFilterType;
1052 }
1053 }
1054
1055
1056
1057
1058 abstract static class Comparators {
1059
1060
1061
1062
1063
1064
1065 static final Comparator<StoreFile> FLUSH_TIME =
1066 Ordering.compound(ImmutableList.of(
1067 Ordering.natural().onResultOf(new GetBulkTime()),
1068 Ordering.natural().onResultOf(new GetSeqId()),
1069 Ordering.natural().onResultOf(new GetPathName())
1070 ));
1071
1072 private static class GetBulkTime implements Function<StoreFile, Long> {
1073 @Override
1074 public Long apply(StoreFile sf) {
1075 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1076 return sf.getBulkLoadTimestamp();
1077 }
1078 }
1079 private static class GetSeqId implements Function<StoreFile, Long> {
1080 @Override
1081 public Long apply(StoreFile sf) {
1082 if (sf.isBulkLoadResult()) return -1L;
1083 return sf.getMaxSequenceId();
1084 }
1085 }
1086 private static class GetPathName implements Function<StoreFile, String> {
1087 @Override
1088 public String apply(StoreFile sf) {
1089 return sf.getPath().getName();
1090 }
1091 }
1092
1093 }
1094 }