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.DataInput;
23 import java.io.FileNotFoundException;
24 import java.io.IOException;
25 import java.nio.ByteBuffer;
26 import java.util.Arrays;
27 import java.util.Collection;
28 import java.util.Collections;
29 import java.util.Comparator;
30 import java.util.Map;
31 import java.util.SortedSet;
32 import java.util.UUID;
33 import java.util.concurrent.atomic.AtomicBoolean;
34 import java.util.regex.Matcher;
35 import java.util.regex.Pattern;
36
37 import org.apache.commons.logging.Log;
38 import org.apache.commons.logging.LogFactory;
39 import org.apache.hadoop.conf.Configuration;
40 import org.apache.hadoop.fs.FileStatus;
41 import org.apache.hadoop.fs.FileSystem;
42 import org.apache.hadoop.fs.Path;
43 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
44 import org.apache.hadoop.hbase.KeyValue;
45 import org.apache.hadoop.hbase.KeyValue.KVComparator;
46 import org.apache.hadoop.hbase.client.Scan;
47 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
48 import org.apache.hadoop.hbase.io.Reference;
49 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
50 import org.apache.hadoop.hbase.io.hfile.Compression;
51 import org.apache.hadoop.hbase.io.hfile.HFile;
52 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
53 import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
54 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
55 import org.apache.hadoop.hbase.util.BloomFilter;
56 import org.apache.hadoop.hbase.util.BloomFilterFactory;
57 import org.apache.hadoop.hbase.util.BloomFilterWriter;
58 import org.apache.hadoop.hbase.util.Bytes;
59 import org.apache.hadoop.hbase.util.FSUtils;
60 import org.apache.hadoop.hbase.util.Writables;
61 import org.apache.hadoop.io.RawComparator;
62 import org.apache.hadoop.io.WritableUtils;
63
64 import com.google.common.base.Function;
65 import com.google.common.collect.ImmutableList;
66 import com.google.common.collect.Ordering;
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81 public class StoreFile {
82 static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
83
84 static final String HFILE_BLOCK_CACHE_SIZE_KEY = "hfile.block.cache.size";
85
86 public static enum BloomType {
87
88
89
90 NONE,
91
92
93
94 ROW,
95
96
97
98 ROWCOL
99 }
100
101
102
103
104 public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
105
106
107 public static final byte[] MAJOR_COMPACTION_KEY =
108 Bytes.toBytes("MAJOR_COMPACTION_KEY");
109
110
111 static final byte[] BLOOM_FILTER_TYPE_KEY =
112 Bytes.toBytes("BLOOM_FILTER_TYPE");
113
114
115 private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
116
117
118 public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
119
120
121
122 public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
123
124 private final FileSystem fs;
125
126
127 private final Path path;
128
129
130 private Reference reference;
131
132
133 private Path referencePath;
134
135
136 private final CacheConfig cacheConf;
137
138
139 private HDFSBlocksDistribution hdfsBlocksDistribution;
140
141
142
143 private long sequenceid = -1;
144
145
146
147 private long maxMemstoreTS = -1;
148
149 public long getMaxMemstoreTS() {
150 return maxMemstoreTS;
151 }
152
153 public void setMaxMemstoreTS(long maxMemstoreTS) {
154 this.maxMemstoreTS = maxMemstoreTS;
155 }
156
157
158
159 private AtomicBoolean majorCompaction = null;
160
161
162 public static final byte[] BULKLOAD_TASK_KEY =
163 Bytes.toBytes("BULKLOAD_SOURCE_TASK");
164 public static final byte[] BULKLOAD_TIME_KEY =
165 Bytes.toBytes("BULKLOAD_TIMESTAMP");
166
167
168
169
170 private Map<byte[], byte[]> metadataMap;
171
172
173
174
175
176
177 private static final Pattern REF_NAME_PARSER =
178 Pattern.compile("^([0-9a-f]+)(?:\\.(.+))?$");
179
180
181 private volatile Reader reader;
182
183
184
185
186
187 private final BloomType cfBloomType;
188
189
190 private long modificationTimeStamp = 0L;
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208 StoreFile(final FileSystem fs,
209 final Path p,
210 final Configuration conf,
211 final CacheConfig cacheConf,
212 final BloomType cfBloomType)
213 throws IOException {
214 this.fs = fs;
215 this.path = p;
216 this.cacheConf = cacheConf;
217 if (isReference(p)) {
218 this.reference = Reference.read(fs, p);
219 this.referencePath = getReferredToFile(this.path);
220 }
221
222 if (BloomFilterFactory.isBloomEnabled(conf)) {
223 this.cfBloomType = cfBloomType;
224 } else {
225 LOG.info("Ignoring bloom filter check for file " + path + ": " +
226 "cfBloomType=" + cfBloomType + " (disabled in config)");
227 this.cfBloomType = BloomType.NONE;
228 }
229
230
231 FileStatus[] stats = FSUtils.listStatus(fs, p, null);
232 if (stats != null && stats.length == 1) {
233 this.modificationTimeStamp = stats[0].getModificationTime();
234 } else {
235 this.modificationTimeStamp = 0;
236 }
237 }
238
239
240
241
242 Path getPath() {
243 return this.path;
244 }
245
246
247
248
249 byte [] getFamily() {
250 return Bytes.toBytes(this.path.getParent().getName());
251 }
252
253
254
255
256
257 boolean isReference() {
258 return this.reference != null;
259 }
260
261
262
263
264
265 public static boolean isReference(final Path p) {
266 return !p.getName().startsWith("_") &&
267 isReference(p, REF_NAME_PARSER.matcher(p.getName()));
268 }
269
270
271
272
273
274
275 public static boolean isReference(final Path p, final Matcher m) {
276 if (m == null || !m.matches()) {
277 LOG.warn("Failed match of store file name " + p.toString());
278 throw new RuntimeException("Failed match of store file name " +
279 p.toString());
280 }
281 return m.groupCount() > 1 && m.group(2) != null;
282 }
283
284
285
286
287
288
289
290
291 static Path getReferredToFile(final Path p) {
292 Matcher m = REF_NAME_PARSER.matcher(p.getName());
293 if (m == null || !m.matches()) {
294 LOG.warn("Failed match of store file name " + p.toString());
295 throw new RuntimeException("Failed match of store file name " +
296 p.toString());
297 }
298
299 String otherRegion = m.group(2);
300
301 Path tableDir = p.getParent().getParent().getParent();
302 String nameStrippedOfSuffix = m.group(1);
303
304
305 return new Path(new Path(new Path(tableDir, otherRegion),
306 p.getParent().getName()), nameStrippedOfSuffix);
307 }
308
309
310
311
312 boolean isMajorCompaction() {
313 if (this.majorCompaction == null) {
314 throw new NullPointerException("This has not been set yet");
315 }
316 return this.majorCompaction.get();
317 }
318
319
320
321
322 public long getMaxSequenceId() {
323 return this.sequenceid;
324 }
325
326 public long getModificationTimeStamp() {
327 return modificationTimeStamp;
328 }
329
330
331
332
333
334
335
336
337
338 public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
339 long max = 0;
340 for (StoreFile sf : sfs) {
341 if (!sf.isBulkLoadResult()) {
342 max = Math.max(max, sf.getMaxMemstoreTS());
343 }
344 }
345 return max;
346 }
347
348
349
350
351
352
353
354
355
356 public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
357 long max = 0;
358 for (StoreFile sf : sfs) {
359 if (!sf.isBulkLoadResult()) {
360 max = Math.max(max, sf.getMaxSequenceId());
361 }
362 }
363 return max;
364 }
365
366
367
368
369
370 boolean isBulkLoadResult() {
371 return metadataMap.containsKey(BULKLOAD_TIME_KEY);
372 }
373
374
375
376
377 public long getBulkLoadTimestamp() {
378 return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
379 }
380
381
382
383
384
385 public HDFSBlocksDistribution getHDFSBlockDistribution() {
386 return this.hdfsBlocksDistribution;
387 }
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402 static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
403 FileSystem fs, Reference reference, Path referencePath) throws IOException {
404 if ( referencePath == null) {
405 return null;
406 }
407
408 FileStatus status = fs.getFileStatus(referencePath);
409 long start = 0;
410 long length = 0;
411
412 if (Reference.isTopFileRegion(reference.getFileRegion())) {
413 start = status.getLen()/2;
414 length = status.getLen() - status.getLen()/2;
415 } else {
416 start = 0;
417 length = status.getLen()/2;
418 }
419 return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
420 }
421
422
423
424
425
426
427
428
429 static public HDFSBlocksDistribution computeHDFSBlockDistribution(
430 FileSystem fs, Path p) throws IOException {
431 if (isReference(p)) {
432 Reference reference = Reference.read(fs, p);
433 Path referencePath = getReferredToFile(p);
434 return computeRefFileHDFSBlockDistribution(fs, reference, referencePath);
435 } else {
436 FileStatus status = fs.getFileStatus(p);
437 long length = status.getLen();
438 return FSUtils.computeHDFSBlocksDistribution(fs, status, 0, length);
439 }
440 }
441
442
443
444
445
446 private void computeHDFSBlockDistribution() throws IOException {
447 if (isReference()) {
448 this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
449 this.fs, this.reference, this.referencePath);
450 } else {
451 FileStatus status = this.fs.getFileStatus(this.path);
452 long length = status.getLen();
453 this.hdfsBlocksDistribution = FSUtils.computeHDFSBlocksDistribution(
454 this.fs, status, 0, length);
455 }
456 }
457
458
459
460
461
462
463
464 private Reader open() throws IOException {
465 if (this.reader != null) {
466 throw new IllegalAccessError("Already open");
467 }
468 if (isReference()) {
469 this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
470 this.cacheConf, this.reference);
471 } else {
472 this.reader = new Reader(this.fs, this.path, this.cacheConf);
473 }
474
475 computeHDFSBlockDistribution();
476
477
478 metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
479
480
481 byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
482 if (b != null) {
483
484
485
486
487
488 this.sequenceid = Bytes.toLong(b);
489 if (isReference()) {
490 if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
491 this.sequenceid += 1;
492 }
493 }
494 }
495 this.reader.setSequenceID(this.sequenceid);
496
497 b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
498 if (b != null) {
499 this.maxMemstoreTS = Bytes.toLong(b);
500 }
501
502 b = metadataMap.get(MAJOR_COMPACTION_KEY);
503 if (b != null) {
504 boolean mc = Bytes.toBoolean(b);
505 if (this.majorCompaction == null) {
506 this.majorCompaction = new AtomicBoolean(mc);
507 } else {
508 this.majorCompaction.set(mc);
509 }
510 } else {
511
512
513 this.majorCompaction = new AtomicBoolean(false);
514 }
515
516 BloomType hfileBloomType = reader.getBloomFilterType();
517 if (cfBloomType != BloomType.NONE) {
518 reader.loadBloomfilter();
519 if (hfileBloomType != cfBloomType) {
520 LOG.info("HFile Bloom filter type for "
521 + reader.getHFileReader().getName() + ": " + hfileBloomType
522 + ", but " + cfBloomType + " specified in column family "
523 + "configuration");
524 }
525 } else if (hfileBloomType != BloomType.NONE) {
526 LOG.info("Bloom filter turned off by CF config for "
527 + reader.getHFileReader().getName());
528 }
529
530 try {
531 byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
532 if (timerangeBytes != null) {
533 this.reader.timeRangeTracker = new TimeRangeTracker();
534 Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
535 }
536 } catch (IllegalArgumentException e) {
537 LOG.error("Error reading timestamp range data from meta -- " +
538 "proceeding without", e);
539 this.reader.timeRangeTracker = null;
540 }
541 return this.reader;
542 }
543
544
545
546
547
548 public Reader createReader() throws IOException {
549 if (this.reader == null) {
550 this.reader = open();
551 }
552 return this.reader;
553 }
554
555
556
557
558
559 public Reader getReader() {
560 return this.reader;
561 }
562
563
564
565
566
567 public synchronized void closeReader(boolean evictOnClose)
568 throws IOException {
569 if (this.reader != null) {
570 this.reader.close(evictOnClose);
571 this.reader = null;
572 }
573 }
574
575
576
577
578
579 public void deleteReader() throws IOException {
580 closeReader(true);
581 this.fs.delete(getPath(), true);
582 }
583
584 @Override
585 public String toString() {
586 return this.path.toString() +
587 (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
588 }
589
590
591
592
593 public String toStringDetailed() {
594 StringBuilder sb = new StringBuilder();
595 sb.append(this.path.toString());
596 sb.append(", isReference=").append(isReference());
597 sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
598 if (isBulkLoadResult()) {
599 sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
600 } else {
601 sb.append(", seqid=").append(getMaxSequenceId());
602 }
603 sb.append(", majorCompaction=").append(isMajorCompaction());
604
605 return sb.toString();
606 }
607
608
609
610
611
612
613
614
615
616 public static Path rename(final FileSystem fs,
617 final Path src,
618 final Path tgt)
619 throws IOException {
620
621 if (!fs.exists(src)) {
622 throw new FileNotFoundException(src.toString());
623 }
624 if (!fs.rename(src, tgt)) {
625 throw new IOException("Failed rename of " + src + " to " + tgt);
626 }
627 return tgt;
628 }
629
630
631
632
633
634
635
636
637
638
639
640 public static Writer createWriter(final FileSystem fs, final Path dir,
641 final int blocksize, Configuration conf, CacheConfig cacheConf)
642 throws IOException {
643 return createWriter(fs, dir, blocksize, null, null, conf, cacheConf,
644 BloomType.NONE, 0);
645 }
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663 public static StoreFile.Writer createWriter(final FileSystem fs,
664 final Path dir,
665 final int blocksize,
666 final Compression.Algorithm algorithm,
667 final KeyValue.KVComparator c,
668 final Configuration conf,
669 final CacheConfig cacheConf,
670 BloomType bloomType,
671 long maxKeyCount)
672 throws IOException {
673
674 if (!fs.exists(dir)) {
675 fs.mkdirs(dir);
676 }
677 Path path = getUniqueFile(fs, dir);
678 if (!BloomFilterFactory.isBloomEnabled(conf)) {
679 bloomType = BloomType.NONE;
680 }
681
682 return new Writer(fs, path, blocksize,
683 algorithm == null? HFile.DEFAULT_COMPRESSION_ALGORITHM: algorithm,
684 conf, cacheConf, c == null ? KeyValue.COMPARATOR: c, bloomType,
685 maxKeyCount);
686 }
687
688
689
690
691
692
693 public static Path getUniqueFile(final FileSystem fs, final Path dir)
694 throws IOException {
695 if (!fs.getFileStatus(dir).isDir()) {
696 throw new IOException("Expecting " + dir.toString() +
697 " to be a directory");
698 }
699 return getRandomFilename(fs, dir);
700 }
701
702
703
704
705
706
707
708
709 static Path getRandomFilename(final FileSystem fs, final Path dir)
710 throws IOException {
711 return getRandomFilename(fs, dir, null);
712 }
713
714
715
716
717
718
719
720
721
722 static Path getRandomFilename(final FileSystem fs,
723 final Path dir,
724 final String suffix)
725 throws IOException {
726 return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
727 + (suffix == null ? "" : suffix));
728 }
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744 static Path split(final FileSystem fs,
745 final Path splitDir,
746 final StoreFile f,
747 final byte [] splitRow,
748 final Reference.Range range)
749 throws IOException {
750
751 Reference r = new Reference(splitRow, range);
752
753
754
755
756 String parentRegionName = f.getPath().getParent().getParent().getName();
757
758
759 Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
760 return r.write(fs, p);
761 }
762
763
764
765
766
767
768 public static class Writer {
769 private final BloomFilterWriter bloomFilterWriter;
770 private final BloomType bloomType;
771 private byte[] lastBloomKey;
772 private int lastBloomKeyOffset, lastBloomKeyLen;
773 private KVComparator kvComparator;
774 private KeyValue lastKv = null;
775
776 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
777
778
779
780
781
782
783 boolean isTimeRangeTrackerSet = false;
784
785 protected HFile.Writer writer;
786
787
788
789
790
791
792
793
794
795
796
797
798
799 public Writer(FileSystem fs, Path path, int blocksize,
800 Compression.Algorithm compress, final Configuration conf,
801 CacheConfig cacheConf,
802 final KVComparator comparator, BloomType bloomType, long maxKeys)
803 throws IOException {
804 writer = HFile.getWriterFactory(conf, cacheConf).createWriter(
805 fs, path, blocksize,
806 compress, comparator.getRawComparator());
807
808 this.kvComparator = comparator;
809
810 bloomFilterWriter = BloomFilterFactory.createBloomAtWrite(conf, cacheConf,
811 bloomType, (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
812 if (bloomFilterWriter != null) {
813 this.bloomType = bloomType;
814 LOG.info("Bloom filter type for " + path + ": " + this.bloomType +
815 ", "+ bloomFilterWriter.getClass().getSimpleName());
816 } else {
817
818 this.bloomType = BloomType.NONE;
819 }
820 }
821
822
823
824
825
826
827
828
829 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
830 throws IOException {
831 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
832 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
833 Bytes.toBytes(majorCompaction));
834 appendTimeRangeMetadata();
835 }
836
837
838
839
840 public void appendTimeRangeMetadata() throws IOException {
841 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
842 }
843
844
845
846
847
848 public void setTimeRangeTracker(final TimeRangeTracker trt) {
849 this.timeRangeTracker = trt;
850 isTimeRangeTrackerSet = true;
851 }
852
853
854
855
856
857
858 public void includeInTimeRangeTracker(final KeyValue kv) {
859 if (!isTimeRangeTrackerSet) {
860 timeRangeTracker.includeTimestamp(kv);
861 }
862 }
863
864
865
866
867
868
869
870 public void includeInTimeRangeTracker(final byte [] key) {
871 if (!isTimeRangeTrackerSet) {
872 timeRangeTracker.includeTimestamp(key);
873 }
874 }
875
876 public void append(final KeyValue kv) throws IOException {
877 if (this.bloomFilterWriter != null) {
878
879 boolean newKey = true;
880 if (this.lastKv != null) {
881 switch(bloomType) {
882 case ROW:
883 newKey = ! kvComparator.matchingRows(kv, lastKv);
884 break;
885 case ROWCOL:
886 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
887 break;
888 case NONE:
889 newKey = false;
890 break;
891 default:
892 throw new IOException("Invalid Bloom filter type: " + bloomType);
893 }
894 }
895 if (newKey) {
896
897
898
899
900
901
902
903
904 byte[] bloomKey;
905 int bloomKeyOffset, bloomKeyLen;
906
907 switch (bloomType) {
908 case ROW:
909 bloomKey = kv.getBuffer();
910 bloomKeyOffset = kv.getRowOffset();
911 bloomKeyLen = kv.getRowLength();
912 break;
913 case ROWCOL:
914
915
916
917 bloomKey = bloomFilterWriter.createBloomKey(kv.getBuffer(),
918 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
919 kv.getQualifierOffset(), kv.getQualifierLength());
920 bloomKeyOffset = 0;
921 bloomKeyLen = bloomKey.length;
922 break;
923 default:
924 throw new IOException("Invalid Bloom filter type: " + bloomType +
925 " (ROW or ROWCOL expected)");
926 }
927 bloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
928 if (lastBloomKey != null
929 && bloomFilterWriter.getComparator().compare(bloomKey,
930 bloomKeyOffset, bloomKeyLen, lastBloomKey,
931 lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
932 throw new IOException("Non-increasing Bloom keys: "
933 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
934 + " after "
935 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
936 lastBloomKeyLen));
937 }
938 lastBloomKey = bloomKey;
939 lastBloomKeyOffset = bloomKeyOffset;
940 lastBloomKeyLen = bloomKeyLen;
941 this.lastKv = kv;
942 }
943 }
944 writer.append(kv);
945 includeInTimeRangeTracker(kv);
946 }
947
948 public Path getPath() {
949 return this.writer.getPath();
950 }
951
952 boolean hasBloom() {
953 return this.bloomFilterWriter != null;
954 }
955
956
957
958
959
960 BloomFilterWriter getBloomWriter() {
961 return bloomFilterWriter;
962 }
963
964 public void close() throws IOException {
965
966 boolean haveBloom = bloomFilterWriter != null &&
967 bloomFilterWriter.getKeyCount() > 0;
968 if (haveBloom) {
969 bloomFilterWriter.compactBloom();
970 writer.addBloomFilter(bloomFilterWriter);
971 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
972 Bytes.toBytes(bloomType.toString()));
973 if (lastBloomKey != null) {
974 writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
975 lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
976 + lastBloomKeyLen));
977 }
978 }
979 writer.close();
980
981
982
983 if (haveBloom && bloomFilterWriter.getMaxKeys() > 0) {
984 StoreFile.LOG.info("Bloom added to HFile ("
985 + getPath() + "): " +
986 bloomFilterWriter.toString().replace("\n", "; "));
987 }
988 }
989
990 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
991 writer.appendFileInfo(key, value);
992 }
993 }
994
995
996
997
998 public static class Reader {
999 static final Log LOG = LogFactory.getLog(Reader.class.getName());
1000
1001 protected BloomFilter bloomFilter = null;
1002 protected BloomType bloomFilterType;
1003 private final HFile.Reader reader;
1004 protected TimeRangeTracker timeRangeTracker = null;
1005 protected long sequenceID = -1;
1006 private byte[] lastBloomKey;
1007
1008 public Reader(FileSystem fs, Path path, CacheConfig cacheConf)
1009 throws IOException {
1010 reader = HFile.createReader(fs, path, cacheConf);
1011 bloomFilterType = BloomType.NONE;
1012 }
1013
1014
1015
1016
1017 Reader() {
1018 this.reader = null;
1019 }
1020
1021 public RawComparator<byte []> getComparator() {
1022 return reader.getComparator();
1023 }
1024
1025
1026
1027
1028
1029
1030
1031
1032 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread) {
1033 return getStoreFileScanner(cacheBlocks, pread, false);
1034 }
1035
1036
1037
1038
1039
1040
1041
1042
1043 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1044 boolean pread, boolean isCompaction) {
1045 return new StoreFileScanner(this, getScanner(cacheBlocks, pread,
1046 isCompaction), !isCompaction);
1047 }
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058 @Deprecated
1059 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1060 return getScanner(cacheBlocks, pread, false);
1061 }
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076 @Deprecated
1077 public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1078 boolean isCompaction) {
1079 return reader.getScanner(cacheBlocks, pread, isCompaction);
1080 }
1081
1082 public void close(boolean evictOnClose) throws IOException {
1083 reader.close(evictOnClose);
1084 }
1085
1086 public boolean shouldSeek(Scan scan, final SortedSet<byte[]> columns) {
1087 return (passesTimerangeFilter(scan) && passesBloomFilter(scan, columns));
1088 }
1089
1090
1091
1092
1093
1094
1095 private boolean passesTimerangeFilter(Scan scan) {
1096 if (timeRangeTracker == null) {
1097 return true;
1098 } else {
1099 return timeRangeTracker.includesTimeRange(scan.getTimeRange());
1100 }
1101 }
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119 private boolean passesBloomFilter(Scan scan,
1120 final SortedSet<byte[]> columns) {
1121
1122
1123 if (!scan.isGetScan()) {
1124 return true;
1125 }
1126
1127 byte[] row = scan.getStartRow();
1128 switch (this.bloomFilterType) {
1129 case ROW:
1130 return passesBloomFilter(row, 0, row.length, null, 0, 0);
1131
1132 case ROWCOL:
1133 if (columns != null && columns.size() == 1) {
1134 byte[] column = columns.first();
1135 return passesBloomFilter(row, 0, row.length, column, 0,
1136 column.length);
1137 }
1138
1139
1140
1141 return true;
1142
1143 default:
1144 return true;
1145 }
1146 }
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160 public boolean passesBloomFilter(byte[] row, int rowOffset, int rowLen,
1161 byte[] col, int colOffset, int colLen) {
1162 if (bloomFilter == null)
1163 return true;
1164
1165 byte[] key;
1166 switch (bloomFilterType) {
1167 case ROW:
1168 if (col != null) {
1169 throw new RuntimeException("Row-only Bloom filter called with " +
1170 "column specified");
1171 }
1172 if (rowOffset != 0 || rowLen != row.length) {
1173 throw new AssertionError("For row-only Bloom filters the row "
1174 + "must occupy the whole array");
1175 }
1176 key = row;
1177 break;
1178
1179 case ROWCOL:
1180 key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1181 colOffset, colLen);
1182 break;
1183
1184 default:
1185 return true;
1186 }
1187
1188
1189
1190 BloomFilter bloomFilter = this.bloomFilter;
1191
1192 if (bloomFilter == null) {
1193 return true;
1194 }
1195
1196
1197 if (reader.getTrailer().getEntryCount() == 0)
1198 return false;
1199
1200 try {
1201 boolean shouldCheckBloom;
1202 ByteBuffer bloom;
1203 if (bloomFilter.supportsAutoLoading()) {
1204 bloom = null;
1205 shouldCheckBloom = true;
1206 } else {
1207 bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1208 true);
1209 shouldCheckBloom = bloom != null;
1210 }
1211
1212 if (shouldCheckBloom) {
1213 boolean exists;
1214
1215
1216
1217
1218 boolean keyIsAfterLast = lastBloomKey != null
1219 && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1220
1221 if (bloomFilterType == BloomType.ROWCOL) {
1222
1223
1224
1225
1226 byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1227 null, 0, 0);
1228
1229 if (keyIsAfterLast
1230 && bloomFilter.getComparator().compare(rowBloomKey,
1231 lastBloomKey) > 0) {
1232 exists = false;
1233 } else {
1234 exists =
1235 this.bloomFilter.contains(key, 0, key.length, bloom) ||
1236 this.bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1237 bloom);
1238 }
1239 } else {
1240 exists = !keyIsAfterLast
1241 && this.bloomFilter.contains(key, 0, key.length, bloom);
1242 }
1243
1244 return exists;
1245 }
1246 } catch (IOException e) {
1247 LOG.error("Error reading bloom filter data -- proceeding without",
1248 e);
1249 setBloomFilterFaulty();
1250 } catch (IllegalArgumentException e) {
1251 LOG.error("Bad bloom filter data -- proceeding without", e);
1252 setBloomFilterFaulty();
1253 }
1254
1255 return true;
1256 }
1257
1258 public Map<byte[], byte[]> loadFileInfo() throws IOException {
1259 Map<byte [], byte []> fi = reader.loadFileInfo();
1260
1261 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1262 if (b != null) {
1263 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1264 }
1265
1266 lastBloomKey = fi.get(LAST_BLOOM_KEY);
1267
1268 return fi;
1269 }
1270
1271 public void loadBloomfilter() {
1272 if (this.bloomFilter != null) {
1273 return;
1274 }
1275
1276 try {
1277 DataInput bloomMeta = reader.getBloomFilterMetadata();
1278 if (bloomMeta != null) {
1279 if (bloomFilterType == BloomType.NONE) {
1280 throw new IOException(
1281 "valid bloom filter type not found in FileInfo");
1282 }
1283
1284 bloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader);
1285 LOG.info("Loaded " + bloomFilterType + " " +
1286 bloomFilter.getClass().getSimpleName() + " metadata for " +
1287 reader.getName());
1288 }
1289 } catch (IOException e) {
1290 LOG.error("Error reading bloom filter meta -- proceeding without", e);
1291 this.bloomFilter = null;
1292 } catch (IllegalArgumentException e) {
1293 LOG.error("Bad bloom filter meta -- proceeding without", e);
1294 this.bloomFilter = null;
1295 }
1296 }
1297
1298
1299
1300
1301
1302
1303
1304
1305 public long getFilterEntries() {
1306 return bloomFilter != null ? bloomFilter.getKeyCount()
1307 : reader.getEntries();
1308 }
1309
1310 public void setBloomFilterFaulty() {
1311 bloomFilter = null;
1312 }
1313
1314 public byte[] getLastKey() {
1315 return reader.getLastKey();
1316 }
1317
1318 public byte[] midkey() throws IOException {
1319 return reader.midkey();
1320 }
1321
1322 public long length() {
1323 return reader.length();
1324 }
1325
1326 public long getTotalUncompressedBytes() {
1327 return reader.getTrailer().getTotalUncompressedBytes();
1328 }
1329
1330 public long getEntries() {
1331 return reader.getEntries();
1332 }
1333
1334 public byte[] getFirstKey() {
1335 return reader.getFirstKey();
1336 }
1337
1338 public long indexSize() {
1339 return reader.indexSize();
1340 }
1341
1342 public BloomType getBloomFilterType() {
1343 return this.bloomFilterType;
1344 }
1345
1346 public long getSequenceID() {
1347 return sequenceID;
1348 }
1349
1350 public void setSequenceID(long sequenceID) {
1351 this.sequenceID = sequenceID;
1352 }
1353
1354 BloomFilter getBloomFilter() {
1355 return bloomFilter;
1356 }
1357
1358 long getUncompressedDataIndexSize() {
1359 return reader.getTrailer().getUncompressedDataIndexSize();
1360 }
1361
1362 public long getTotalBloomSize() {
1363 if (bloomFilter == null)
1364 return 0;
1365 return bloomFilter.getByteSize();
1366 }
1367
1368 public int getHFileVersion() {
1369 return reader.getTrailer().getVersion();
1370 }
1371
1372 HFile.Reader getHFileReader() {
1373 return reader;
1374 }
1375
1376 void disableBloomFilterForTesting() {
1377 bloomFilter = null;
1378 }
1379 }
1380
1381
1382
1383
1384 abstract static class Comparators {
1385
1386
1387
1388
1389
1390
1391 static final Comparator<StoreFile> FLUSH_TIME =
1392 Ordering.compound(ImmutableList.of(
1393 Ordering.natural().onResultOf(new GetBulkTime()),
1394 Ordering.natural().onResultOf(new GetSeqId()),
1395 Ordering.natural().onResultOf(new GetPathName())
1396 ));
1397
1398 private static class GetBulkTime implements Function<StoreFile, Long> {
1399 @Override
1400 public Long apply(StoreFile sf) {
1401 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1402 return sf.getBulkLoadTimestamp();
1403 }
1404 }
1405 private static class GetSeqId implements Function<StoreFile, Long> {
1406 @Override
1407 public Long apply(StoreFile sf) {
1408 if (sf.isBulkLoadResult()) return -1L;
1409 return sf.getMaxSequenceId();
1410 }
1411 }
1412 private static class GetPathName implements Function<StoreFile, String> {
1413 @Override
1414 public String apply(StoreFile sf) {
1415 return sf.getPath().getName();
1416 }
1417 }
1418
1419
1420
1421
1422 static final Comparator<StoreFile> FILE_SIZE =
1423 Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1424 @Override
1425 public Long apply(StoreFile sf) {
1426 return sf.getReader().length();
1427 }
1428 });
1429 }
1430 }