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_MAX_KEYS = "io.storefile.bloom.max.keys";
87 static final String IO_STOREFILE_BLOOM_ENABLED = "io.storefile.bloom.enabled";
88 static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
89
90 public static enum BloomType {
91
92
93
94 NONE,
95
96
97
98 ROW,
99
100
101
102 ROWCOL
103 }
104
105
106 public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
107
108 public static final byte [] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
109
110 static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
111
112 static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
113
114
115 static final String BLOOM_FILTER_META_KEY = "BLOOM_FILTER_META";
116
117 static final String BLOOM_FILTER_DATA_KEY = "BLOOM_FILTER_DATA";
118
119
120
121 public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
122
123
124 private static BlockCache hfileBlockCache = null;
125
126 private final FileSystem fs;
127
128 private final Path path;
129
130 private Reference reference;
131
132 private Path referencePath;
133
134 private boolean blockcache;
135
136 private boolean inMemory;
137
138
139
140 private long sequenceid = -1;
141
142
143
144 private AtomicBoolean majorCompaction = null;
145
146
147 public static final byte[] BULKLOAD_TASK_KEY =
148 Bytes.toBytes("BULKLOAD_SOURCE_TASK");
149 public static final byte[] BULKLOAD_TIME_KEY =
150 Bytes.toBytes("BULKLOAD_TIMESTAMP");
151
152
153
154
155 private Map<byte[], byte[]> metadataMap;
156
157
158
159
160
161
162 private static final Pattern REF_NAME_PARSER =
163 Pattern.compile("^(\\d+)(?:\\.(.+))?$");
164
165
166 private volatile Reader reader;
167
168
169 private final static Random rand = new Random();
170 private final Configuration conf;
171 private final BloomType bloomType;
172
173
174
175
176
177
178
179
180
181
182
183
184
185 StoreFile(final FileSystem fs,
186 final Path p,
187 final boolean blockcache,
188 final Configuration conf,
189 final BloomType bt,
190 final boolean inMemory)
191 throws IOException {
192 this.conf = conf;
193 this.fs = fs;
194 this.path = p;
195 this.blockcache = blockcache;
196 this.inMemory = inMemory;
197 if (isReference(p)) {
198 this.reference = Reference.read(fs, p);
199 this.referencePath = getReferredToFile(this.path);
200 }
201
202
203 if (conf.getBoolean(IO_STOREFILE_BLOOM_ENABLED, true)) {
204 this.bloomType = bt;
205 } else {
206 this.bloomType = BloomType.NONE;
207 LOG.info("Ignoring bloom filter check for file (disabled in config)");
208 }
209 }
210
211
212
213
214 Path getPath() {
215 return this.path;
216 }
217
218
219
220
221 byte [] getFamily() {
222 return Bytes.toBytes(this.path.getParent().getName());
223 }
224
225
226
227
228
229 boolean isReference() {
230 return this.reference != null;
231 }
232
233
234
235
236
237 public static boolean isReference(final Path p) {
238 return !p.getName().startsWith("_") &&
239 isReference(p, REF_NAME_PARSER.matcher(p.getName()));
240 }
241
242
243
244
245
246
247 public static boolean isReference(final Path p, final Matcher m) {
248 if (m == null || !m.matches()) {
249 LOG.warn("Failed match of store file name " + p.toString());
250 throw new RuntimeException("Failed match of store file name " +
251 p.toString());
252 }
253 return m.groupCount() > 1 && m.group(2) != null;
254 }
255
256
257
258
259
260
261
262
263 static Path getReferredToFile(final Path p) {
264 Matcher m = REF_NAME_PARSER.matcher(p.getName());
265 if (m == null || !m.matches()) {
266 LOG.warn("Failed match of store file name " + p.toString());
267 throw new RuntimeException("Failed match of store file name " +
268 p.toString());
269 }
270
271 String otherRegion = m.group(2);
272
273 Path tableDir = p.getParent().getParent().getParent();
274 String nameStrippedOfSuffix = m.group(1);
275
276
277 return new Path(new Path(new Path(tableDir, otherRegion),
278 p.getParent().getName()), nameStrippedOfSuffix);
279 }
280
281
282
283
284 boolean isMajorCompaction() {
285 if (this.majorCompaction == null) {
286 throw new NullPointerException("This has not been set yet");
287 }
288 return this.majorCompaction.get();
289 }
290
291
292
293
294 public long getMaxSequenceId() {
295 return this.sequenceid;
296 }
297
298
299
300
301
302
303
304
305
306 public static long getMaxSequenceIdInList(List<StoreFile> sfs) {
307 long max = 0;
308 for (StoreFile sf : sfs) {
309 if (!sf.isBulkLoadResult()) {
310 max = Math.max(max, sf.getMaxSequenceId());
311 }
312 }
313 return max;
314 }
315
316
317
318
319
320 boolean isBulkLoadResult() {
321 return metadataMap.containsKey(BULKLOAD_TIME_KEY);
322 }
323
324
325
326
327 public long getBulkLoadTimestamp() {
328 return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
329 }
330
331
332
333
334
335
336
337 public static synchronized BlockCache getBlockCache(Configuration conf) {
338 if (hfileBlockCache != null) return hfileBlockCache;
339
340 float cachePercentage = conf.getFloat(HFILE_BLOCK_CACHE_SIZE_KEY, 0.2f);
341
342 if (cachePercentage == 0L) return null;
343 if (cachePercentage > 1.0) {
344 throw new IllegalArgumentException(HFILE_BLOCK_CACHE_SIZE_KEY +
345 " must be between 0.0 and 1.0, not > 1.0");
346 }
347
348
349 MemoryUsage mu = ManagementFactory.getMemoryMXBean().getHeapMemoryUsage();
350 long cacheSize = (long)(mu.getMax() * cachePercentage);
351 LOG.info("Allocating LruBlockCache with maximum size " +
352 StringUtils.humanReadableInt(cacheSize));
353 hfileBlockCache = new LruBlockCache(cacheSize, DEFAULT_BLOCKSIZE_SMALL);
354 return hfileBlockCache;
355 }
356
357
358
359
360 public BlockCache getBlockCache() {
361 return blockcache ? getBlockCache(conf) : null;
362 }
363
364
365
366
367
368
369
370 private Reader open() throws IOException {
371 if (this.reader != null) {
372 throw new IllegalAccessError("Already open");
373 }
374 if (isReference()) {
375 this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
376 getBlockCache(), this.reference);
377 } else {
378 this.reader = new Reader(this.fs, this.path, getBlockCache(),
379 this.inMemory);
380 }
381
382 metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
383
384 byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
385 if (b != null) {
386
387
388
389
390
391 this.sequenceid = Bytes.toLong(b);
392 if (isReference()) {
393 if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
394 this.sequenceid += 1;
395 }
396 }
397 }
398 this.reader.setSequenceID(this.sequenceid);
399
400 b = metadataMap.get(MAJOR_COMPACTION_KEY);
401 if (b != null) {
402 boolean mc = Bytes.toBoolean(b);
403 if (this.majorCompaction == null) {
404 this.majorCompaction = new AtomicBoolean(mc);
405 } else {
406 this.majorCompaction.set(mc);
407 }
408 } else {
409
410
411 this.majorCompaction = new AtomicBoolean(false);
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 BloomFilter bloom = null;
696 BloomType bt = BloomType.NONE;
697
698 if (bloomType != BloomType.NONE && conf != null) {
699 float err = conf.getFloat(IO_STOREFILE_BLOOM_ERROR_RATE, (float)0.01);
700
701
702
703
704 if (bloomType == BloomType.ROWCOL) {
705 err /= 2;
706 }
707 int maxFold = conf.getInt(IO_STOREFILE_BLOOM_MAX_FOLD, 7);
708 int tooBig = conf.getInt(IO_STOREFILE_BLOOM_MAX_KEYS, 128*1000*1000);
709
710 if (maxKeys < tooBig) {
711 try {
712 bloom = new ByteBloomFilter(maxKeys, err,
713 Hash.getHashType(conf), maxFold);
714 bloom.allocBloom();
715 bt = bloomType;
716 } catch (IllegalArgumentException iae) {
717 LOG.warn(String.format(
718 "Parse error while creating bloom for %s (%d, %f)",
719 path, maxKeys, err), iae);
720 bloom = null;
721 bt = BloomType.NONE;
722 }
723 } else {
724 if (LOG.isDebugEnabled()) {
725 LOG.debug("Skipping bloom filter because max keysize too large: "
726 + maxKeys);
727 }
728 }
729 }
730
731 this.bloomFilter = bloom;
732 this.bloomType = bt;
733 }
734
735
736
737
738
739
740
741
742 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
743 throws IOException {
744 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
745 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
746 Bytes.toBytes(majorCompaction));
747 appendTimeRangeMetadata();
748 }
749
750
751
752
753 public void appendTimeRangeMetadata() throws IOException {
754 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
755 }
756
757
758
759
760
761 public void setTimeRangeTracker(final TimeRangeTracker trt) {
762 this.timeRangeTracker = trt;
763 isTimeRangeTrackerSet = true;
764 }
765
766
767
768
769
770
771
772 public void includeInTimeRangeTracker(final KeyValue kv) {
773 if (!isTimeRangeTrackerSet) {
774 timeRangeTracker.includeTimestamp(kv);
775 }
776 }
777
778
779
780
781
782
783
784 public void includeInTimeRangeTracker(final byte [] key) {
785 if (!isTimeRangeTrackerSet) {
786 timeRangeTracker.includeTimestamp(key);
787 }
788 }
789
790 public void append(final KeyValue kv) throws IOException {
791 if (this.bloomFilter != null) {
792
793 boolean newKey = true;
794 if (this.lastKv != null) {
795 switch(bloomType) {
796 case ROW:
797 newKey = ! kvComparator.matchingRows(kv, lastKv);
798 break;
799 case ROWCOL:
800 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
801 break;
802 case NONE:
803 newKey = false;
804 }
805 }
806 if (newKey) {
807
808
809
810
811
812
813
814
815 switch (bloomType) {
816 case ROW:
817 this.bloomFilter.add(kv.getBuffer(), kv.getRowOffset(),
818 kv.getRowLength());
819 break;
820 case ROWCOL:
821
822 int ro = kv.getRowOffset();
823 int rl = kv.getRowLength();
824 int qo = kv.getQualifierOffset();
825 int ql = kv.getQualifierLength();
826 byte [] result = new byte[rl + ql];
827 System.arraycopy(kv.getBuffer(), ro, result, 0, rl);
828 System.arraycopy(kv.getBuffer(), qo, result, rl, ql);
829 this.bloomFilter.add(result);
830 break;
831 default:
832 }
833 this.lastKv = kv;
834 }
835 }
836 writer.append(kv);
837 includeInTimeRangeTracker(kv);
838 }
839
840 public Path getPath() {
841 return this.writer.getPath();
842 }
843
844 boolean hasBloom() {
845 return this.bloomFilter != null;
846 }
847
848 public void append(final byte [] key, final byte [] value) throws IOException {
849 if (this.bloomFilter != null) {
850
851 if (this.lastByteArray == null || !Arrays.equals(key, lastByteArray)) {
852 this.bloomFilter.add(key);
853 this.lastByteArray = key;
854 }
855 }
856 writer.append(key, value);
857 includeInTimeRangeTracker(key);
858 }
859
860 public void close() throws IOException {
861
862 if (this.bloomFilter != null && this.bloomFilter.getKeyCount() > 0) {
863 bloomFilter.compactBloom();
864 if (this.bloomFilter.getMaxKeys() > 0) {
865 int b = this.bloomFilter.getByteSize();
866 int k = this.bloomFilter.getKeyCount();
867 int m = this.bloomFilter.getMaxKeys();
868 StoreFile.LOG.info("Bloom added to HFile (" +
869 getPath() + "): " + StringUtils.humanReadableInt(b) + ", " +
870 k + "/" + m + " (" + NumberFormat.getPercentInstance().format(
871 ((double)k) / ((double)m)) + ")");
872 }
873 writer.appendMetaBlock(BLOOM_FILTER_META_KEY, bloomFilter.getMetaWriter());
874 writer.appendMetaBlock(BLOOM_FILTER_DATA_KEY, bloomFilter.getDataWriter());
875 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY, Bytes.toBytes(bloomType.toString()));
876 }
877 writer.close();
878 }
879
880 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
881 writer.appendFileInfo(key, value);
882 }
883 }
884
885
886
887
888 public static class Reader {
889 static final Log LOG = LogFactory.getLog(Reader.class.getName());
890
891 protected BloomFilter bloomFilter = null;
892 protected BloomType bloomFilterType;
893 private final HFile.Reader reader;
894 protected TimeRangeTracker timeRangeTracker = null;
895 protected long sequenceID = -1;
896
897 public Reader(FileSystem fs, Path path, BlockCache blockCache, boolean inMemory)
898 throws IOException {
899 reader = new HFile.Reader(fs, path, blockCache, inMemory);
900 bloomFilterType = BloomType.NONE;
901 }
902
903 public RawComparator<byte []> getComparator() {
904 return reader.getComparator();
905 }
906
907
908
909
910
911
912
913
914 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
915 return new StoreFileScanner(this, getScanner(cacheBlocks, pread));
916 }
917
918
919
920
921
922
923
924
925
926
927 @Deprecated
928 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
929 return reader.getScanner(cacheBlocks, pread);
930 }
931
932 public void close() throws IOException {
933 reader.close();
934 }
935
936 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
937 return (passesTimerangeFilter(scan) && passesBloomFilter(scan,columns));
938 }
939
940
941
942
943
944
945 private boolean passesTimerangeFilter(Scan scan) {
946 if (timeRangeTracker == null) {
947 return true;
948 } else {
949 return timeRangeTracker.includesTimeRange(scan.getTimeRange());
950 }
951 }
952
953 private boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
954 BloomFilter bm = this.bloomFilter;
955 if (bm == null || !scan.isGetScan()) {
956 return true;
957 }
958 byte[] row = scan.getStartRow();
959 byte[] key;
960 switch (this.bloomFilterType) {
961 case ROW:
962 key = row;
963 break;
964 case ROWCOL:
965 if (columns != null && columns.size() == 1) {
966 byte[] col = columns.first();
967 key = Bytes.add(row, col);
968 break;
969 }
970
971 default:
972 return true;
973 }
974
975 try {
976 ByteBuffer bloom = reader.getMetaBlock(BLOOM_FILTER_DATA_KEY, true);
977 if (bloom != null) {
978 if (this.bloomFilterType == BloomType.ROWCOL) {
979
980
981
982
983 return bm.contains(key, bloom) ||
984 bm.contains(row, bloom);
985 }
986 else {
987 return bm.contains(key, bloom);
988 }
989 }
990 } catch (IOException e) {
991 LOG.error("Error reading bloom filter data -- proceeding without",
992 e);
993 setBloomFilterFaulty();
994 } catch (IllegalArgumentException e) {
995 LOG.error("Bad bloom filter data -- proceeding without", e);
996 setBloomFilterFaulty();
997 }
998
999 return true;
1000 }
1001
1002 public Map<byte[], byte[]> loadFileInfo() throws IOException {
1003 Map<byte [], byte []> fi = reader.loadFileInfo();
1004
1005 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1006 if (b != null) {
1007 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1008 }
1009
1010 return fi;
1011 }
1012
1013 public void loadBloomfilter() {
1014 if (this.bloomFilter != null) {
1015 return;
1016 }
1017
1018 try {
1019 ByteBuffer b = reader.getMetaBlock(BLOOM_FILTER_META_KEY, false);
1020 if (b != null) {
1021 if (bloomFilterType == BloomType.NONE) {
1022 throw new IOException("valid bloom filter type not found in FileInfo");
1023 }
1024
1025
1026 this.bloomFilter = new ByteBloomFilter(b);
1027 LOG.info("Loaded " + (bloomFilterType== BloomType.ROW? "row":"col")
1028 + " bloom filter metadata for " + reader.getName());
1029 }
1030 } catch (IOException e) {
1031 LOG.error("Error reading bloom filter meta -- proceeding without", e);
1032 this.bloomFilter = null;
1033 } catch (IllegalArgumentException e) {
1034 LOG.error("Bad bloom filter meta -- proceeding without", e);
1035 this.bloomFilter = null;
1036 }
1037 }
1038
1039 public int getFilterEntries() {
1040 return (this.bloomFilter != null) ? this.bloomFilter.getKeyCount()
1041 : reader.getFilterEntries();
1042 }
1043
1044 public ByteBuffer getMetaBlock(String bloomFilterDataKey, boolean cacheBlock) throws IOException {
1045 return reader.getMetaBlock(bloomFilterDataKey, cacheBlock);
1046 }
1047
1048 public void setBloomFilterFaulty() {
1049 bloomFilter = null;
1050 }
1051
1052 public byte[] getLastKey() {
1053 return reader.getLastKey();
1054 }
1055
1056 public byte[] midkey() throws IOException {
1057 return reader.midkey();
1058 }
1059
1060 public long length() {
1061 return reader.length();
1062 }
1063
1064 public int getEntries() {
1065 return reader.getEntries();
1066 }
1067
1068 public byte[] getFirstKey() {
1069 return reader.getFirstKey();
1070 }
1071
1072 public long indexSize() {
1073 return reader.indexSize();
1074 }
1075
1076 public BloomType getBloomFilterType() {
1077 return this.bloomFilterType;
1078 }
1079
1080 public long getSequenceID() {
1081 return sequenceID;
1082 }
1083
1084 public void setSequenceID(long sequenceID) {
1085 this.sequenceID = sequenceID;
1086 }
1087 }
1088
1089
1090
1091
1092 abstract static class Comparators {
1093
1094
1095
1096
1097
1098
1099 static final Comparator<StoreFile> FLUSH_TIME =
1100 Ordering.compound(ImmutableList.of(
1101 Ordering.natural().onResultOf(new GetBulkTime()),
1102 Ordering.natural().onResultOf(new GetSeqId()),
1103 Ordering.natural().onResultOf(new GetPathName())
1104 ));
1105
1106 private static class GetBulkTime implements Function<StoreFile, Long> {
1107 @Override
1108 public Long apply(StoreFile sf) {
1109 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1110 return sf.getBulkLoadTimestamp();
1111 }
1112 }
1113 private static class GetSeqId implements Function<StoreFile, Long> {
1114 @Override
1115 public Long apply(StoreFile sf) {
1116 if (sf.isBulkLoadResult()) return -1L;
1117 return sf.getMaxSequenceId();
1118 }
1119 }
1120 private static class GetPathName implements Function<StoreFile, String> {
1121 @Override
1122 public String apply(StoreFile sf) {
1123 return sf.getPath().getName();
1124 }
1125 }
1126
1127 }
1128 }