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 int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
698
699 this.bloomFilter = new ByteBloomFilter(maxKeys, err,
700 Hash.getHashType(conf), maxFold);
701 this.bloomFilter.allocBloom();
702 this.bloomType = bloomType;
703 } else {
704 this.bloomFilter = null;
705 this.bloomType = BloomType.NONE;
706 }
707 }
708
709
710
711
712
713
714
715
716 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
717 throws IOException {
718 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
719 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
720 Bytes.toBytes(majorCompaction));
721 appendTimeRangeMetadata();
722 }
723
724
725
726
727 public void appendTimeRangeMetadata() throws IOException {
728 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
729 }
730
731
732
733
734
735 public void setTimeRangeTracker(final TimeRangeTracker trt) {
736 this.timeRangeTracker = trt;
737 isTimeRangeTrackerSet = true;
738 }
739
740
741
742
743
744
745
746 public void includeInTimeRangeTracker(final KeyValue kv) {
747 if (!isTimeRangeTrackerSet) {
748 timeRangeTracker.includeTimestamp(kv);
749 }
750 }
751
752
753
754
755
756
757
758 public void includeInTimeRangeTracker(final byte [] key) {
759 if (!isTimeRangeTrackerSet) {
760 timeRangeTracker.includeTimestamp(key);
761 }
762 }
763
764 public void append(final KeyValue kv) throws IOException {
765 if (this.bloomFilter != null) {
766
767 boolean newKey = true;
768 if (this.lastKv != null) {
769 switch(bloomType) {
770 case ROW:
771 newKey = ! kvComparator.matchingRows(kv, lastKv);
772 break;
773 case ROWCOL:
774 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
775 break;
776 case NONE:
777 newKey = false;
778 }
779 }
780 if (newKey) {
781
782
783
784
785
786
787
788
789 switch (bloomType) {
790 case ROW:
791 this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
792 kv.getRowLength());
793 break;
794 case ROWCOL:
795
796 int ro = kv.getRowOffset();
797 int rl = kv.getRowLength();
798 int qo = kv.getQualifierOffset();
799 int ql = kv.getQualifierLength();
800 byte [] result = new byte[rl + ql];
801 System.arraycopy(kv.getBuffer(), ro, result, 0, rl);
802 System.arraycopy(kv.getBuffer(), qo, result, rl, ql);
803
804 this.bloomFilter.add(result);
805 break;
806 default:
807 }
808 this.lastKv = kv;
809 }
810 }
811 writer.append(kv);
812 includeInTimeRangeTracker(kv);
813 }
814
815 public Path getPath() {
816 return this.writer.getPath();
817 }
818
819 public void append(final byte [] key, final byte [] value) throws IOException {
820 if (this.bloomFilter != null) {
821
822 if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) {
823 this.bloomFilter.add(key);
824 this.lastByteArray = key;
825 }
826 }
827 writer.append(key, value);
828 includeInTimeRangeTracker(key);
829 }
830
831 public void close() throws IOException {
832
833 if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
834 bloomFilter.compactBloom();
835 if (this.bloomFilter.getMaxKeys() > 0) {
836 int b = this.bloomFilter.getByteSize();
837 int k = this.bloomFilter.getKeyCount();
838 int m = this.bloomFilter.getMaxKeys();
839 StoreFile.LOG.info("Bloom added to HFile. " + b + "B, " +
840 k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
841 ((double)k) / ((double)m)) + ")");
842 }
843 writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter());
844 writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter());
845 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
846 }
847 writer.close();
848 }
849
850 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
851 writer.appendFileInfo(key, value);
852 }
853 }
854
855
856
857
858 public static class Reader {
859 static final Log LOG = LogFactory.getLog(Reader.class.getName());
860
861 protected BloomFilter bloomFilter = null;
862 protected BloomType bloomFilterType;
863 private final HFile.Reader reader;
864 protected TimeRangeTracker timeRangeTracker = null;
865
866 public Reader(FileSystem fs, Path path, BlockCache blockCache, boolean inMemory)
867 throws IOException {
868 reader = new HFile.Reader(fs, path, blockCache, inMemory);
869 bloomFilterType = BloomType.NONE;
870 }
871
872 public RawComparator<byte []> getComparator() {
873 return reader.getComparator();
874 }
875
876
877
878
879
880
881
882
883 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
884 return new StoreFileScanner(this, getScanner(cacheBlocks, pread));
885 }
886
887
888
889
890
891
892
893
894
895
896 @Deprecated
897 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
898 return reader.getScanner(cacheBlocks, pread);
899 }
900
901 public void close() throws IOException {
902 reader.close();
903 }
904
905 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
906 return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns));
907 }
908
909
910
911
912
913
914 private boolean passesTimerangeFilter(Scan scan) {
915 if (timeRangeTracker == null) {
916 return true;
917 } else {
918 return timeRangeTracker.includesTimeRange(scan.getTimeRange());
919 }
920 }
921
922 private boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
923 if (this.bloomFilter == null || !scan.isGetScan()) {
924 return true;
925 }
926 byte[] row = scan.getStartRow();
927 byte[] key;
928 switch (this.bloomFilterType) {
929 case ROW:
930 key = row;
931 break;
932 case ROWCOL:
933 if (columns != null && columns.size() == 1) {
934 byte[] col = columns.first();
935 key = Bytes.add(row, col);
936 break;
937 }
938
939 default:
940 return true;
941 }
942
943 try {
944 ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
945 if (bloom != null) {
946 return this.bloomFilter.contains(key, bloom);
947 }
948 } catch (IOException e) {
949 LOG.error("Error reading bloom filter data -- proceeding without",
950 e);
951 setBloomFilterFaulty();
952 } catch (IllegalArgumentException e) {
953 LOG.error("Bad bloom filter data -- proceeding without", e);
954 setBloomFilterFaulty();
955 }
956
957 return true;
958 }
959
960 public Map<byte[], byte[]> loadFileInfo() throws IOException {
961 Map<byte [], byte []> fi = reader.loadFileInfo();
962
963 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
964 if (b != null) {
965 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
966 }
967
968 return fi;
969 }
970
971 public void loadBloomfilter() {
972 if (this.bloomFilter != null) {
973 return;
974 }
975
976 try {
977 ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false);
978 if (b != null) {
979 if (bloomFilterType == BloomType.NONE) {
980 throw new IOException("valid bloom filter type not found in FileInfo");
981 }
982
983
984 this.bloomFilter = new ByteBloomFilter(b);
985 LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col")
986 + " bloom filter metadata for " + reader.getName());
987 }
988 } catch (IOException e) {
989 LOG.error("Error reading bloom filter meta -- proceeding without", e);
990 this.bloomFilter = null;
991 } catch (IllegalArgumentException e) {
992 LOG.error("Bad bloom filter meta -- proceeding without", e);
993 this.bloomFilter = null;
994 }
995 }
996
997 public int getFilterEntries() {
998 return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
999 : reader.getFilterEntries();
1000 }
1001
1002 public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException {
1003 return reader.getMetaBlock(bloomFilterDataKey, cacheBlock);
1004 }
1005
1006 public void setBloomFilterFaulty() {
1007 bloomFilter = null;
1008 }
1009
1010 public byte[] getLastKey() {
1011 return reader.getLastKey();
1012 }
1013
1014 public byte[] midkey() throws IOException {
1015 return reader.midkey();
1016 }
1017
1018 public long length() {
1019 return reader.length();
1020 }
1021
1022 public int getEntries() {
1023 return reader.getEntries();
1024 }
1025
1026 public byte[] getFirstKey() {
1027 return reader.getFirstKey();
1028 }
1029
1030 public long indexSize() {
1031 return reader.indexSize();
1032 }
1033
1034 public BloomType getBloomFilterType() {
1035 return this.bloomFilterType;
1036 }
1037 }
1038
1039
1040
1041
1042 abstract static class Comparators {
1043
1044
1045
1046
1047
1048
1049 static final Comparator<StoreFile> FLUSH_TIME =
1050 Ordering.compound(ImmutableList.of(
1051 Ordering.natural().onResultOf(new GetBulkTime()),
1052 Ordering.natural().onResultOf(new GetSeqId()),
1053 Ordering.natural().onResultOf(new GetPathName())
1054 ));
1055
1056 private static class GetBulkTime implements Function<StoreFile, Long> {
1057 @Override
1058 public Long apply(StoreFile sf) {
1059 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1060 return sf.getBulkLoadTimestamp();
1061 }
1062 }
1063 private static class GetSeqId implements Function<StoreFile, Long> {
1064 @Override
1065 public Long apply(StoreFile sf) {
1066 if (sf.isBulkLoadResult()) return -1L;
1067 return sf.getMaxSequenceId();
1068 }
1069 }
1070 private static class GetPathName implements Function<StoreFile, String> {
1071 @Override
1072 public String apply(StoreFile sf) {
1073 return sf.getPath().getName();
1074 }
1075 }
1076
1077 }
1078 }