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.FSDataInputStream;
41 import org.apache.hadoop.fs.FileStatus;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
46 import org.apache.hadoop.hbase.HTableDescriptor;
47 import org.apache.hadoop.hbase.KeyValue;
48 import org.apache.hadoop.hbase.KeyValue.KVComparator;
49 import org.apache.hadoop.hbase.client.Scan;
50 import org.apache.hadoop.hbase.fs.HFileSystem;
51 import org.apache.hadoop.hbase.io.HFileLink;
52 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
53 import org.apache.hadoop.hbase.io.Reference;
54 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
55 import org.apache.hadoop.hbase.io.hfile.BlockType;
56 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
57 import org.apache.hadoop.hbase.io.hfile.Compression;
58 import org.apache.hadoop.hbase.io.hfile.HFile;
59 import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
60 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
61 import org.apache.hadoop.hbase.io.hfile.HFileWriterV1;
62 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
63 import org.apache.hadoop.hbase.io.hfile.NoOpDataBlockEncoder;
64 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
65 import org.apache.hadoop.hbase.regionserver.metrics.SchemaConfigured;
66 import org.apache.hadoop.hbase.util.BloomFilter;
67 import org.apache.hadoop.hbase.util.BloomFilterFactory;
68 import org.apache.hadoop.hbase.util.BloomFilterWriter;
69 import org.apache.hadoop.hbase.util.Bytes;
70 import org.apache.hadoop.hbase.util.ChecksumType;
71 import org.apache.hadoop.hbase.util.FSUtils;
72 import org.apache.hadoop.hbase.util.Writables;
73 import org.apache.hadoop.io.RawComparator;
74 import org.apache.hadoop.io.WritableUtils;
75
76 import com.google.common.base.Function;
77 import com.google.common.base.Preconditions;
78 import com.google.common.collect.ImmutableList;
79 import com.google.common.collect.Ordering;
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94 public class StoreFile extends SchemaConfigured {
95 static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
96
97 public static enum BloomType {
98
99
100
101 NONE,
102
103
104
105 ROW,
106
107
108
109 ROWCOL
110 }
111
112
113
114
115 public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
116
117
118 public static final byte[] MAJOR_COMPACTION_KEY =
119 Bytes.toBytes("MAJOR_COMPACTION_KEY");
120
121
122 public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
123 Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
124
125
126 static final byte[] BLOOM_FILTER_TYPE_KEY =
127 Bytes.toBytes("BLOOM_FILTER_TYPE");
128
129
130 public static final byte[] DELETE_FAMILY_COUNT =
131 Bytes.toBytes("DELETE_FAMILY_COUNT");
132
133
134 private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
135
136
137 public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
138
139
140 public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
141
142
143
144 public static final int DEFAULT_BLOCKSIZE_SMALL = 8 * 1024;
145
146 private final FileSystem fs;
147
148
149 private final Path path;
150
151
152 private Reference reference;
153
154
155 private Path referencePath;
156
157
158 private HFileLink link;
159
160
161 private final CacheConfig cacheConf;
162
163
164 private final HFileDataBlockEncoder dataBlockEncoder;
165
166
167 private HDFSBlocksDistribution hdfsBlocksDistribution;
168
169
170
171 private long sequenceid = -1;
172
173
174
175 private long maxMemstoreTS = -1;
176
177 public long getMaxMemstoreTS() {
178 return maxMemstoreTS;
179 }
180
181 public void setMaxMemstoreTS(long maxMemstoreTS) {
182 this.maxMemstoreTS = maxMemstoreTS;
183 }
184
185
186
187 private AtomicBoolean majorCompaction = null;
188
189
190
191 private boolean excludeFromMinorCompaction = false;
192
193
194 public static final byte[] BULKLOAD_TASK_KEY =
195 Bytes.toBytes("BULKLOAD_SOURCE_TASK");
196 public static final byte[] BULKLOAD_TIME_KEY =
197 Bytes.toBytes("BULKLOAD_TIMESTAMP");
198
199
200
201
202 private Map<byte[], byte[]> metadataMap;
203
204
205
206
207
208 public static final String HFILE_NAME_REGEX = "[0-9a-f]+(?:_SeqId_[0-9]+_)?";
209
210
211 private static final Pattern HFILE_NAME_PATTERN =
212 Pattern.compile("^(" + HFILE_NAME_REGEX + ")");
213
214
215
216
217
218
219
220
221 private static final Pattern REF_NAME_PATTERN =
222 Pattern.compile(String.format("^(%s|%s)\\.(.+)$",
223 HFILE_NAME_REGEX, HFileLink.LINK_NAME_REGEX));
224
225
226 private volatile Reader reader;
227
228
229
230
231
232 private final BloomType cfBloomType;
233
234
235 private long modificationTimeStamp = 0L;
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254 public StoreFile(final FileSystem fs,
255 final Path p,
256 final Configuration conf,
257 final CacheConfig cacheConf,
258 final BloomType cfBloomType,
259 final HFileDataBlockEncoder dataBlockEncoder)
260 throws IOException {
261 this.fs = fs;
262 this.path = p;
263 this.cacheConf = cacheConf;
264 this.dataBlockEncoder =
265 dataBlockEncoder == null ? NoOpDataBlockEncoder.INSTANCE
266 : dataBlockEncoder;
267
268 if (HFileLink.isHFileLink(p)) {
269 this.link = new HFileLink(conf, p);
270 LOG.debug("Store file " + p + " is a link");
271 } else if (isReference(p)) {
272 this.reference = Reference.read(fs, p);
273 this.referencePath = getReferredToFile(this.path);
274 if (HFileLink.isHFileLink(this.referencePath)) {
275 this.link = new HFileLink(conf, this.referencePath);
276 }
277 LOG.debug("Store file " + p + " is a " + reference.getFileRegion() +
278 " reference to " + this.referencePath);
279 } else if (!isHFile(p)) {
280 throw new IOException("path=" + path + " doesn't look like a valid StoreFile");
281 }
282
283 if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
284 this.cfBloomType = cfBloomType;
285 } else {
286 LOG.info("Ignoring bloom filter check for file " + path + ": " +
287 "cfBloomType=" + cfBloomType + " (disabled in config)");
288 this.cfBloomType = BloomType.NONE;
289 }
290
291
292 FileStatus[] stats = FSUtils.listStatus(fs, p, null);
293 if (stats != null && stats.length == 1) {
294 this.modificationTimeStamp = stats[0].getModificationTime();
295 } else {
296 this.modificationTimeStamp = 0;
297 }
298
299 SchemaMetrics.configureGlobally(conf);
300 }
301
302
303
304
305 public Path getPath() {
306 return this.path;
307 }
308
309
310
311
312 byte [] getFamily() {
313 return Bytes.toBytes(this.path.getParent().getName());
314 }
315
316
317
318
319
320 boolean isReference() {
321 return this.reference != null;
322 }
323
324
325
326
327 boolean isLink() {
328 return this.link != null && this.reference == null;
329 }
330
331 private static boolean isHFile(final Path path) {
332 Matcher m = HFILE_NAME_PATTERN.matcher(path.getName());
333 return m.matches() && m.groupCount() > 0;
334 }
335
336
337
338
339
340 public static boolean isReference(final Path p) {
341 return isReference(p.getName());
342 }
343
344
345
346
347
348 public static boolean isReference(final String name) {
349 Matcher m = REF_NAME_PATTERN.matcher(name);
350 return m.matches() && m.groupCount() > 1;
351 }
352
353
354
355
356
357
358
359
360 public static Path getReferredToFile(final Path p) {
361 Matcher m = REF_NAME_PATTERN.matcher(p.getName());
362 if (m == null || !m.matches()) {
363 LOG.warn("Failed match of store file name " + p.toString());
364 throw new IllegalArgumentException("Failed match of store file name " +
365 p.toString());
366 }
367
368 String otherRegion = m.group(2);
369
370 Path tableDir = p.getParent().getParent().getParent();
371 String nameStrippedOfSuffix = m.group(1);
372 LOG.debug("reference '" + p + "' to region=" + otherRegion + " hfile=" + nameStrippedOfSuffix);
373
374
375
376 return new Path(new Path(new Path(tableDir, otherRegion),
377 p.getParent().getName()), nameStrippedOfSuffix);
378 }
379
380
381
382
383 boolean isMajorCompaction() {
384 if (this.majorCompaction == null) {
385 throw new NullPointerException("This has not been set yet");
386 }
387 return this.majorCompaction.get();
388 }
389
390
391
392
393 boolean excludeFromMinorCompaction() {
394 return this.excludeFromMinorCompaction;
395 }
396
397
398
399
400 public long getMaxSequenceId() {
401 return this.sequenceid;
402 }
403
404 public long getModificationTimeStamp() {
405 return modificationTimeStamp;
406 }
407
408
409
410
411
412
413
414
415
416 public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
417 long max = 0;
418 for (StoreFile sf : sfs) {
419 if (!sf.isBulkLoadResult()) {
420 max = Math.max(max, sf.getMaxMemstoreTS());
421 }
422 }
423 return max;
424 }
425
426
427
428
429
430
431
432
433
434 public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
435 long max = 0;
436 for (StoreFile sf : sfs) {
437 if (!sf.isBulkLoadResult()) {
438 max = Math.max(max, sf.getMaxSequenceId());
439 }
440 }
441 return max;
442 }
443
444
445
446
447
448 boolean isBulkLoadResult() {
449 return metadataMap.containsKey(BULKLOAD_TIME_KEY);
450 }
451
452
453
454
455 public long getBulkLoadTimestamp() {
456 return Bytes.toLong(metadataMap.get(BULKLOAD_TIME_KEY));
457 }
458
459
460
461
462
463 public HDFSBlocksDistribution getHDFSBlockDistribution() {
464 return this.hdfsBlocksDistribution;
465 }
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480 static private HDFSBlocksDistribution computeRefFileHDFSBlockDistribution(
481 FileSystem fs, Reference reference, FileStatus status) throws IOException {
482 if (status == null) {
483 return null;
484 }
485
486 long start = 0;
487 long length = 0;
488
489 if (Reference.isTopFileRegion(reference.getFileRegion())) {
490 start = status.getLen()/2;
491 length = status.getLen() - status.getLen()/2;
492 } else {
493 start = 0;
494 length = status.getLen()/2;
495 }
496 return FSUtils.computeHDFSBlocksDistribution(fs, status, start, length);
497 }
498
499
500
501
502 private void computeHDFSBlockDistribution() throws IOException {
503 if (isReference()) {
504 FileStatus status;
505 if (this.link != null) {
506 status = this.link.getFileStatus(fs);
507 } else {
508 status = fs.getFileStatus(this.referencePath);
509 }
510 this.hdfsBlocksDistribution = computeRefFileHDFSBlockDistribution(
511 this.fs, this.reference, status);
512 } else {
513 FileStatus status;
514 if (isLink()) {
515 status = link.getFileStatus(fs);
516 } else {
517 status = this.fs.getFileStatus(path);
518 }
519 long length = status.getLen();
520 this.hdfsBlocksDistribution = FSUtils.computeHDFSBlocksDistribution(
521 this.fs, status, 0, length);
522 }
523 }
524
525
526
527
528
529
530
531 private Reader open() throws IOException {
532 if (this.reader != null) {
533 throw new IllegalAccessError("Already open");
534 }
535 if (isReference()) {
536 if (this.link != null) {
537 this.reader = new HalfStoreFileReader(this.fs, this.referencePath, this.link,
538 this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
539 } else {
540 this.reader = new HalfStoreFileReader(this.fs, this.referencePath,
541 this.cacheConf, this.reference, dataBlockEncoder.getEncodingInCache());
542 }
543 } else if (isLink()) {
544 long size = link.getFileStatus(fs).getLen();
545 this.reader = new Reader(this.fs, this.path, link, size, this.cacheConf,
546 dataBlockEncoder.getEncodingInCache(), true);
547 } else {
548 this.reader = new Reader(this.fs, this.path, this.cacheConf,
549 dataBlockEncoder.getEncodingInCache());
550 }
551
552 if (isSchemaConfigured()) {
553 SchemaConfigured.resetSchemaMetricsConf(reader);
554 passSchemaMetricsTo(reader);
555 }
556
557 computeHDFSBlockDistribution();
558
559
560 metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
561
562
563 byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
564 if (b != null) {
565
566
567
568
569
570 this.sequenceid = Bytes.toLong(b);
571 if (isReference()) {
572 if (Reference.isTopFileRegion(this.reference.getFileRegion())) {
573 this.sequenceid += 1;
574 }
575 }
576 }
577 this.reader.setSequenceID(this.sequenceid);
578
579 b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
580 if (b != null) {
581 this.maxMemstoreTS = Bytes.toLong(b);
582 }
583
584 b = metadataMap.get(MAJOR_COMPACTION_KEY);
585 if (b != null) {
586 boolean mc = Bytes.toBoolean(b);
587 if (this.majorCompaction == null) {
588 this.majorCompaction = new AtomicBoolean(mc);
589 } else {
590 this.majorCompaction.set(mc);
591 }
592 } else {
593
594
595 this.majorCompaction = new AtomicBoolean(false);
596 }
597
598 b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
599 this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
600
601 BloomType hfileBloomType = reader.getBloomFilterType();
602 if (cfBloomType != BloomType.NONE) {
603 reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
604 if (hfileBloomType != cfBloomType) {
605 LOG.info("HFile Bloom filter type for "
606 + reader.getHFileReader().getName() + ": " + hfileBloomType
607 + ", but " + cfBloomType + " specified in column family "
608 + "configuration");
609 }
610 } else if (hfileBloomType != BloomType.NONE) {
611 LOG.info("Bloom filter turned off by CF config for "
612 + reader.getHFileReader().getName());
613 }
614
615
616 reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
617
618 try {
619 byte [] timerangeBytes = metadataMap.get(TIMERANGE_KEY);
620 if (timerangeBytes != null) {
621 this.reader.timeRangeTracker = new TimeRangeTracker();
622 Writables.copyWritable(timerangeBytes, this.reader.timeRangeTracker);
623 }
624 } catch (IllegalArgumentException e) {
625 LOG.error("Error reading timestamp range data from meta -- " +
626 "proceeding without", e);
627 this.reader.timeRangeTracker = null;
628 }
629 return this.reader;
630 }
631
632
633
634
635
636 public Reader createReader() throws IOException {
637 if (this.reader == null) {
638 try {
639 this.reader = open();
640 } catch (IOException e) {
641 try {
642 this.closeReader(true);
643 } catch (IOException ee) {
644 }
645 throw e;
646 }
647
648 }
649 return this.reader;
650 }
651
652
653
654
655
656 public Reader getReader() {
657 return this.reader;
658 }
659
660
661
662
663
664 public synchronized void closeReader(boolean evictOnClose)
665 throws IOException {
666 if (this.reader != null) {
667 this.reader.close(evictOnClose);
668 this.reader = null;
669 }
670 }
671
672
673
674
675
676 public void deleteReader() throws IOException {
677 closeReader(true);
678 this.fs.delete(getPath(), true);
679 }
680
681 @Override
682 public String toString() {
683 return this.path.toString() +
684 (isReference()? "-" + this.referencePath + "-" + reference.toString(): "");
685 }
686
687
688
689
690 public String toStringDetailed() {
691 StringBuilder sb = new StringBuilder();
692 sb.append(this.path.toString());
693 sb.append(", isReference=").append(isReference());
694 sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
695 if (isBulkLoadResult()) {
696 sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
697 } else {
698 sb.append(", seqid=").append(getMaxSequenceId());
699 }
700 sb.append(", majorCompaction=").append(isMajorCompaction());
701
702 return sb.toString();
703 }
704
705
706
707
708
709
710
711
712
713 public static Path rename(final FileSystem fs,
714 final Path src,
715 final Path tgt)
716 throws IOException {
717
718 if (!fs.exists(src)) {
719 throw new FileNotFoundException(src.toString());
720 }
721 if (!fs.rename(src, tgt)) {
722 throw new IOException("Failed rename of " + src + " to " + tgt);
723 }
724 return tgt;
725 }
726
727 public static class WriterBuilder {
728 private final Configuration conf;
729 private final CacheConfig cacheConf;
730 private final FileSystem fs;
731 private final int blockSize;
732
733 private Compression.Algorithm compressAlgo =
734 HFile.DEFAULT_COMPRESSION_ALGORITHM;
735 private HFileDataBlockEncoder dataBlockEncoder =
736 NoOpDataBlockEncoder.INSTANCE;
737 private KeyValue.KVComparator comparator = KeyValue.COMPARATOR;
738 private BloomType bloomType = BloomType.NONE;
739 private long maxKeyCount = 0;
740 private Path dir;
741 private Path filePath;
742 private ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
743 private int bytesPerChecksum = HFile.DEFAULT_BYTES_PER_CHECKSUM;
744
745 public WriterBuilder(Configuration conf, CacheConfig cacheConf,
746 FileSystem fs, int blockSize) {
747 this.conf = conf;
748 this.cacheConf = cacheConf;
749 this.fs = fs;
750 this.blockSize = blockSize;
751 }
752
753
754
755
756
757
758
759
760 public WriterBuilder withOutputDir(Path dir) {
761 Preconditions.checkNotNull(dir);
762 this.dir = dir;
763 return this;
764 }
765
766
767
768
769
770
771 public WriterBuilder withFilePath(Path filePath) {
772 Preconditions.checkNotNull(filePath);
773 this.filePath = filePath;
774 return this;
775 }
776
777 public WriterBuilder withCompression(Compression.Algorithm compressAlgo) {
778 Preconditions.checkNotNull(compressAlgo);
779 this.compressAlgo = compressAlgo;
780 return this;
781 }
782
783 public WriterBuilder withDataBlockEncoder(HFileDataBlockEncoder encoder) {
784 Preconditions.checkNotNull(encoder);
785 this.dataBlockEncoder = encoder;
786 return this;
787 }
788
789 public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
790 Preconditions.checkNotNull(comparator);
791 this.comparator = comparator;
792 return this;
793 }
794
795 public WriterBuilder withBloomType(BloomType bloomType) {
796 Preconditions.checkNotNull(bloomType);
797 this.bloomType = bloomType;
798 return this;
799 }
800
801
802
803
804
805 public WriterBuilder withMaxKeyCount(long maxKeyCount) {
806 this.maxKeyCount = maxKeyCount;
807 return this;
808 }
809
810
811
812
813
814 public WriterBuilder withChecksumType(ChecksumType checksumType) {
815 this.checksumType = checksumType;
816 return this;
817 }
818
819
820
821
822
823 public WriterBuilder withBytesPerChecksum(int bytesPerChecksum) {
824 this.bytesPerChecksum = bytesPerChecksum;
825 return this;
826 }
827
828
829
830
831
832
833 public Writer build() throws IOException {
834 if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
835 throw new IllegalArgumentException("Either specify parent directory " +
836 "or file path");
837 }
838
839 if (dir == null) {
840 dir = filePath.getParent();
841 }
842
843 if (!fs.exists(dir)) {
844 fs.mkdirs(dir);
845 }
846
847 if (filePath == null) {
848 filePath = getUniqueFile(fs, dir);
849 if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
850 bloomType = BloomType.NONE;
851 }
852 }
853
854 if (compressAlgo == null) {
855 compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
856 }
857 if (comparator == null) {
858 comparator = KeyValue.COMPARATOR;
859 }
860 return new Writer(fs, filePath, blockSize, compressAlgo, dataBlockEncoder,
861 conf, cacheConf, comparator, bloomType, maxKeyCount, checksumType,
862 bytesPerChecksum);
863 }
864 }
865
866
867
868
869
870
871 public static Path getUniqueFile(final FileSystem fs, final Path dir)
872 throws IOException {
873 if (!fs.getFileStatus(dir).isDir()) {
874 throw new IOException("Expecting " + dir.toString() +
875 " to be a directory");
876 }
877 return getRandomFilename(fs, dir);
878 }
879
880
881
882
883
884
885
886
887 static Path getRandomFilename(final FileSystem fs, final Path dir)
888 throws IOException {
889 return getRandomFilename(fs, dir, null);
890 }
891
892
893
894
895
896
897
898
899
900 static Path getRandomFilename(final FileSystem fs,
901 final Path dir,
902 final String suffix)
903 throws IOException {
904 return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
905 + (suffix == null ? "" : suffix));
906 }
907
908
909
910
911
912
913 public static boolean validateStoreFileName(String fileName) {
914 if (HFileLink.isHFileLink(fileName))
915 return true;
916 if (isReference(fileName))
917 return true;
918 return !fileName.contains("-");
919 }
920
921
922
923
924
925
926
927
928
929
930
931
932
933 static Path split(final FileSystem fs,
934 final Path splitDir,
935 final StoreFile f,
936 final byte [] splitRow,
937 final Reference.Range range)
938 throws IOException {
939
940 Reference r = new Reference(splitRow, range);
941
942
943
944
945 String parentRegionName = f.getPath().getParent().getParent().getName();
946
947
948 Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
949 return r.write(fs, p);
950 }
951
952
953
954
955
956
957 public static class Writer {
958 private final BloomFilterWriter generalBloomFilterWriter;
959 private final BloomFilterWriter deleteFamilyBloomFilterWriter;
960 private final BloomType bloomType;
961 private byte[] lastBloomKey;
962 private int lastBloomKeyOffset, lastBloomKeyLen;
963 private KVComparator kvComparator;
964 private KeyValue lastKv = null;
965 private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
966 private KeyValue lastDeleteFamilyKV = null;
967 private long deleteFamilyCnt = 0;
968
969 protected HFileDataBlockEncoder dataBlockEncoder;
970
971
972 protected ChecksumType checksumType;
973
974
975 protected int bytesPerChecksum;
976
977 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
978
979
980
981
982
983
984 boolean isTimeRangeTrackerSet = false;
985
986 protected HFile.Writer writer;
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003 private Writer(FileSystem fs, Path path, int blocksize,
1004 Compression.Algorithm compress,
1005 HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
1006 CacheConfig cacheConf,
1007 final KVComparator comparator, BloomType bloomType, long maxKeys,
1008 final ChecksumType checksumType, final int bytesPerChecksum)
1009 throws IOException {
1010 this.dataBlockEncoder = dataBlockEncoder != null ?
1011 dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
1012 writer = HFile.getWriterFactory(conf, cacheConf)
1013 .withPath(fs, path)
1014 .withBlockSize(blocksize)
1015 .withCompression(compress)
1016 .withDataBlockEncoder(dataBlockEncoder)
1017 .withComparator(comparator.getRawComparator())
1018 .withChecksumType(checksumType)
1019 .withBytesPerChecksum(bytesPerChecksum)
1020 .create();
1021
1022 this.kvComparator = comparator;
1023
1024 generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
1025 conf, cacheConf, bloomType,
1026 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1027
1028 if (generalBloomFilterWriter != null) {
1029 this.bloomType = bloomType;
1030 LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
1031 + generalBloomFilterWriter.getClass().getSimpleName());
1032 } else {
1033
1034 this.bloomType = BloomType.NONE;
1035 }
1036
1037
1038
1039 if (this.bloomType != BloomType.ROWCOL) {
1040 this.deleteFamilyBloomFilterWriter = BloomFilterFactory
1041 .createDeleteBloomAtWrite(conf, cacheConf,
1042 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1043 } else {
1044 deleteFamilyBloomFilterWriter = null;
1045 }
1046 if (deleteFamilyBloomFilterWriter != null) {
1047 LOG.info("Delete Family Bloom filter type for " + path + ": "
1048 + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
1049 }
1050 this.checksumType = checksumType;
1051 this.bytesPerChecksum = bytesPerChecksum;
1052 }
1053
1054
1055
1056
1057
1058
1059
1060
1061 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
1062 throws IOException {
1063 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
1064 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
1065 Bytes.toBytes(majorCompaction));
1066 appendTrackedTimestampsToMetadata();
1067 }
1068
1069
1070
1071
1072 public void appendTrackedTimestampsToMetadata() throws IOException {
1073 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
1074 appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
1075 }
1076
1077
1078
1079
1080
1081 public void setTimeRangeTracker(final TimeRangeTracker trt) {
1082 this.timeRangeTracker = trt;
1083 isTimeRangeTrackerSet = true;
1084 }
1085
1086
1087
1088
1089
1090
1091
1092
1093 public void trackTimestamps(final KeyValue kv) {
1094 if (KeyValue.Type.Put.getCode() == kv.getType()) {
1095 earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
1096 }
1097 if (!isTimeRangeTrackerSet) {
1098 timeRangeTracker.includeTimestamp(kv);
1099 }
1100 }
1101
1102 private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
1103 if (this.generalBloomFilterWriter != null) {
1104
1105 boolean newKey = true;
1106 if (this.lastKv != null) {
1107 switch(bloomType) {
1108 case ROW:
1109 newKey = ! kvComparator.matchingRows(kv, lastKv);
1110 break;
1111 case ROWCOL:
1112 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
1113 break;
1114 case NONE:
1115 newKey = false;
1116 break;
1117 default:
1118 throw new IOException("Invalid Bloom filter type: " + bloomType +
1119 " (ROW or ROWCOL expected)");
1120 }
1121 }
1122 if (newKey) {
1123
1124
1125
1126
1127
1128
1129
1130
1131 byte[] bloomKey;
1132 int bloomKeyOffset, bloomKeyLen;
1133
1134 switch (bloomType) {
1135 case ROW:
1136 bloomKey = kv.getBuffer();
1137 bloomKeyOffset = kv.getRowOffset();
1138 bloomKeyLen = kv.getRowLength();
1139 break;
1140 case ROWCOL:
1141
1142
1143
1144 bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
1145 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
1146 kv.getQualifierOffset(), kv.getQualifierLength());
1147 bloomKeyOffset = 0;
1148 bloomKeyLen = bloomKey.length;
1149 break;
1150 default:
1151 throw new IOException("Invalid Bloom filter type: " + bloomType +
1152 " (ROW or ROWCOL expected)");
1153 }
1154 generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
1155 if (lastBloomKey != null
1156 && generalBloomFilterWriter.getComparator().compare(bloomKey,
1157 bloomKeyOffset, bloomKeyLen, lastBloomKey,
1158 lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
1159 throw new IOException("Non-increasing Bloom keys: "
1160 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
1161 + " after "
1162 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
1163 lastBloomKeyLen));
1164 }
1165 lastBloomKey = bloomKey;
1166 lastBloomKeyOffset = bloomKeyOffset;
1167 lastBloomKeyLen = bloomKeyLen;
1168 this.lastKv = kv;
1169 }
1170 }
1171 }
1172
1173 private void appendDeleteFamilyBloomFilter(final KeyValue kv)
1174 throws IOException {
1175 if (!kv.isDeleteFamily()) {
1176 return;
1177 }
1178
1179
1180 deleteFamilyCnt++;
1181 if (null != this.deleteFamilyBloomFilterWriter) {
1182 boolean newKey = true;
1183 if (lastDeleteFamilyKV != null) {
1184 newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
1185 }
1186 if (newKey) {
1187 this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
1188 kv.getRowOffset(), kv.getRowLength());
1189 this.lastDeleteFamilyKV = kv;
1190 }
1191 }
1192 }
1193
1194 public void append(final KeyValue kv) throws IOException {
1195 appendGeneralBloomfilter(kv);
1196 appendDeleteFamilyBloomFilter(kv);
1197 writer.append(kv);
1198 trackTimestamps(kv);
1199 }
1200
1201 public Path getPath() {
1202 return this.writer.getPath();
1203 }
1204
1205 boolean hasGeneralBloom() {
1206 return this.generalBloomFilterWriter != null;
1207 }
1208
1209
1210
1211
1212
1213
1214 BloomFilterWriter getGeneralBloomWriter() {
1215 return generalBloomFilterWriter;
1216 }
1217
1218 private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
1219 boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
1220 if (haveBloom) {
1221 bfw.compactBloom();
1222 }
1223 return haveBloom;
1224 }
1225
1226 private boolean closeGeneralBloomFilter() throws IOException {
1227 boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
1228
1229
1230 if (hasGeneralBloom) {
1231 writer.addGeneralBloomFilter(generalBloomFilterWriter);
1232 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
1233 Bytes.toBytes(bloomType.toString()));
1234 if (lastBloomKey != null) {
1235 writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1236 lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1237 + lastBloomKeyLen));
1238 }
1239 }
1240 return hasGeneralBloom;
1241 }
1242
1243 private boolean closeDeleteFamilyBloomFilter() throws IOException {
1244 boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1245
1246
1247 if (hasDeleteFamilyBloom) {
1248 writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1249 }
1250
1251
1252
1253 writer.appendFileInfo(DELETE_FAMILY_COUNT,
1254 Bytes.toBytes(this.deleteFamilyCnt));
1255
1256 return hasDeleteFamilyBloom;
1257 }
1258
1259 public void close() throws IOException {
1260 boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1261 boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1262
1263 writer.close();
1264
1265
1266
1267 StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
1268 + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
1269 + " was added to HFile (" + getPath() + ") ");
1270
1271 }
1272
1273 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1274 writer.appendFileInfo(key, value);
1275 }
1276
1277
1278 HFile.Writer getHFileWriter() {
1279 return writer;
1280 }
1281 }
1282
1283
1284
1285
1286 public static class Reader extends SchemaConfigured {
1287 static final Log LOG = LogFactory.getLog(Reader.class.getName());
1288
1289 protected BloomFilter generalBloomFilter = null;
1290 protected BloomFilter deleteFamilyBloomFilter = null;
1291 protected BloomType bloomFilterType;
1292 private final HFile.Reader reader;
1293 protected TimeRangeTracker timeRangeTracker = null;
1294 protected long sequenceID = -1;
1295 private byte[] lastBloomKey;
1296 private long deleteFamilyCnt = -1;
1297
1298 public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
1299 DataBlockEncoding preferredEncodingInCache) throws IOException {
1300 super(path);
1301 reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
1302 preferredEncodingInCache);
1303 bloomFilterType = BloomType.NONE;
1304 }
1305
1306 public Reader(FileSystem fs, Path path, HFileLink hfileLink, long size,
1307 CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
1308 boolean closeIStream) throws IOException {
1309 super(path);
1310
1311 FSDataInputStream in = hfileLink.open(fs);
1312 FSDataInputStream inNoChecksum = in;
1313 if (fs instanceof HFileSystem) {
1314 FileSystem noChecksumFs = ((HFileSystem)fs).getNoChecksumFs();
1315 inNoChecksum = hfileLink.open(noChecksumFs);
1316 }
1317
1318 reader = HFile.createReaderWithEncoding(fs, path, in, inNoChecksum,
1319 size, cacheConf, preferredEncodingInCache, closeIStream);
1320 bloomFilterType = BloomType.NONE;
1321 }
1322
1323
1324
1325
1326 Reader() {
1327 this.reader = null;
1328 }
1329
1330 public RawComparator<byte []> getComparator() {
1331 return reader.getComparator();
1332 }
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1343 boolean pread) {
1344 return getStoreFileScanner(cacheBlocks, pread, false);
1345 }
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1356 boolean pread,
1357 boolean isCompaction) {
1358 return new StoreFileScanner(this,
1359 getScanner(cacheBlocks, pread,
1360 isCompaction), !isCompaction);
1361 }
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372 @Deprecated
1373 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1374 return getScanner(cacheBlocks, pread, false);
1375 }
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390 @Deprecated
1391 public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1392 boolean isCompaction) {
1393 return reader.getScanner(cacheBlocks, pread, isCompaction);
1394 }
1395
1396 public void close(boolean evictOnClose) throws IOException {
1397 reader.close(evictOnClose);
1398 }
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408 boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1409 if (timeRangeTracker == null) {
1410 return true;
1411 } else {
1412 return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1413 timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1414 }
1415 }
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433 boolean passesBloomFilter(Scan scan,
1434 final SortedSet<byte[]> columns) {
1435
1436
1437 if (!scan.isGetScan()) {
1438 return true;
1439 }
1440
1441 byte[] row = scan.getStartRow();
1442 switch (this.bloomFilterType) {
1443 case ROW:
1444 return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1445
1446 case ROWCOL:
1447 if (columns != null && columns.size() == 1) {
1448 byte[] column = columns.first();
1449 return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1450 column.length);
1451 }
1452
1453
1454
1455 return true;
1456
1457 default:
1458 return true;
1459 }
1460 }
1461
1462 public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1463 int rowLen) {
1464
1465
1466 BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1467
1468
1469 if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1470 return false;
1471 }
1472
1473 if (bloomFilter == null) {
1474 return true;
1475 }
1476
1477 try {
1478 if (!bloomFilter.supportsAutoLoading()) {
1479 return true;
1480 }
1481 return bloomFilter.contains(row, rowOffset, rowLen, null);
1482 } catch (IllegalArgumentException e) {
1483 LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1484 e);
1485 setDeleteFamilyBloomFilterFaulty();
1486 }
1487
1488 return true;
1489 }
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503 public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1504 int rowLen, byte[] col, int colOffset, int colLen) {
1505 if (generalBloomFilter == null)
1506 return true;
1507
1508 byte[] key;
1509 switch (bloomFilterType) {
1510 case ROW:
1511 if (col != null) {
1512 throw new RuntimeException("Row-only Bloom filter called with " +
1513 "column specified");
1514 }
1515 if (rowOffset != 0 || rowLen != row.length) {
1516 throw new AssertionError("For row-only Bloom filters the row "
1517 + "must occupy the whole array");
1518 }
1519 key = row;
1520 break;
1521
1522 case ROWCOL:
1523 key = generalBloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1524 colOffset, colLen);
1525 break;
1526
1527 default:
1528 return true;
1529 }
1530
1531
1532
1533 BloomFilter bloomFilter = this.generalBloomFilter;
1534
1535 if (bloomFilter == null) {
1536 return true;
1537 }
1538
1539
1540 if (reader.getTrailer().getEntryCount() == 0)
1541 return false;
1542
1543 try {
1544 boolean shouldCheckBloom;
1545 ByteBuffer bloom;
1546 if (bloomFilter.supportsAutoLoading()) {
1547 bloom = null;
1548 shouldCheckBloom = true;
1549 } else {
1550 bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1551 true);
1552 shouldCheckBloom = bloom != null;
1553 }
1554
1555 if (shouldCheckBloom) {
1556 boolean exists;
1557
1558
1559
1560
1561 boolean keyIsAfterLast = lastBloomKey != null
1562 && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1563
1564 if (bloomFilterType == BloomType.ROWCOL) {
1565
1566
1567
1568
1569 byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1570 null, 0, 0);
1571
1572 if (keyIsAfterLast
1573 && bloomFilter.getComparator().compare(rowBloomKey,
1574 lastBloomKey) > 0) {
1575 exists = false;
1576 } else {
1577 exists =
1578 bloomFilter.contains(key, 0, key.length, bloom) ||
1579 bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1580 bloom);
1581 }
1582 } else {
1583 exists = !keyIsAfterLast
1584 && bloomFilter.contains(key, 0, key.length, bloom);
1585 }
1586
1587 getSchemaMetrics().updateBloomMetrics(exists);
1588 return exists;
1589 }
1590 } catch (IOException e) {
1591 LOG.error("Error reading bloom filter data -- proceeding without",
1592 e);
1593 setGeneralBloomFilterFaulty();
1594 } catch (IllegalArgumentException e) {
1595 LOG.error("Bad bloom filter data -- proceeding without", e);
1596 setGeneralBloomFilterFaulty();
1597 }
1598
1599 return true;
1600 }
1601
1602 public Map<byte[], byte[]> loadFileInfo() throws IOException {
1603 Map<byte [], byte []> fi = reader.loadFileInfo();
1604
1605 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1606 if (b != null) {
1607 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1608 }
1609
1610 lastBloomKey = fi.get(LAST_BLOOM_KEY);
1611 byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1612 if (cnt != null) {
1613 deleteFamilyCnt = Bytes.toLong(cnt);
1614 }
1615
1616 return fi;
1617 }
1618
1619 public void loadBloomfilter() {
1620 this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1621 this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1622 }
1623
1624 private void loadBloomfilter(BlockType blockType) {
1625 try {
1626 if (blockType == BlockType.GENERAL_BLOOM_META) {
1627 if (this.generalBloomFilter != null)
1628 return;
1629
1630 DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1631 if (bloomMeta != null) {
1632
1633 if (bloomFilterType == BloomType.NONE) {
1634 throw new IOException(
1635 "valid bloom filter type not found in FileInfo");
1636 } else {
1637 generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1638 reader);
1639 LOG.info("Loaded " + bloomFilterType.toString() + " ("
1640 + generalBloomFilter.getClass().getSimpleName()
1641 + ") metadata for " + reader.getName());
1642 }
1643 }
1644 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1645 if (this.deleteFamilyBloomFilter != null)
1646 return;
1647
1648 DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1649 if (bloomMeta != null) {
1650 deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1651 bloomMeta, reader);
1652 LOG.info("Loaded Delete Family Bloom ("
1653 + deleteFamilyBloomFilter.getClass().getSimpleName()
1654 + ") metadata for " + reader.getName());
1655 }
1656 } else {
1657 throw new RuntimeException("Block Type: " + blockType.toString()
1658 + "is not supported for Bloom filter");
1659 }
1660 } catch (IOException e) {
1661 LOG.error("Error reading bloom filter meta for " + blockType
1662 + " -- proceeding without", e);
1663 setBloomFilterFaulty(blockType);
1664 } catch (IllegalArgumentException e) {
1665 LOG.error("Bad bloom filter meta " + blockType
1666 + " -- proceeding without", e);
1667 setBloomFilterFaulty(blockType);
1668 }
1669 }
1670
1671 private void setBloomFilterFaulty(BlockType blockType) {
1672 if (blockType == BlockType.GENERAL_BLOOM_META) {
1673 setGeneralBloomFilterFaulty();
1674 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1675 setDeleteFamilyBloomFilterFaulty();
1676 }
1677 }
1678
1679
1680
1681
1682
1683
1684
1685
1686 public long getFilterEntries() {
1687 return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1688 : reader.getEntries();
1689 }
1690
1691 public void setGeneralBloomFilterFaulty() {
1692 generalBloomFilter = null;
1693 }
1694
1695 public void setDeleteFamilyBloomFilterFaulty() {
1696 this.deleteFamilyBloomFilter = null;
1697 }
1698
1699 public byte[] getLastKey() {
1700 return reader.getLastKey();
1701 }
1702
1703 public byte[] midkey() throws IOException {
1704 return reader.midkey();
1705 }
1706
1707 public long length() {
1708 return reader.length();
1709 }
1710
1711 public long getTotalUncompressedBytes() {
1712 return reader.getTrailer().getTotalUncompressedBytes();
1713 }
1714
1715 public long getEntries() {
1716 return reader.getEntries();
1717 }
1718
1719 public long getDeleteFamilyCnt() {
1720 return deleteFamilyCnt;
1721 }
1722
1723 public byte[] getFirstKey() {
1724 return reader.getFirstKey();
1725 }
1726
1727 public long indexSize() {
1728 return reader.indexSize();
1729 }
1730
1731 public String getColumnFamilyName() {
1732 return reader.getColumnFamilyName();
1733 }
1734
1735 public BloomType getBloomFilterType() {
1736 return this.bloomFilterType;
1737 }
1738
1739 public long getSequenceID() {
1740 return sequenceID;
1741 }
1742
1743 public void setSequenceID(long sequenceID) {
1744 this.sequenceID = sequenceID;
1745 }
1746
1747 BloomFilter getGeneralBloomFilter() {
1748 return generalBloomFilter;
1749 }
1750
1751 long getUncompressedDataIndexSize() {
1752 return reader.getTrailer().getUncompressedDataIndexSize();
1753 }
1754
1755 public long getTotalBloomSize() {
1756 if (generalBloomFilter == null)
1757 return 0;
1758 return generalBloomFilter.getByteSize();
1759 }
1760
1761 public int getHFileVersion() {
1762 return reader.getTrailer().getMajorVersion();
1763 }
1764
1765 HFile.Reader getHFileReader() {
1766 return reader;
1767 }
1768
1769 void disableBloomFilterForTesting() {
1770 generalBloomFilter = null;
1771 this.deleteFamilyBloomFilter = null;
1772 }
1773
1774 public long getMaxTimestamp() {
1775 return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.maximumTimestamp;
1776 }
1777
1778 @Override
1779 public void schemaConfigurationChanged() {
1780 passSchemaMetricsTo((SchemaConfigured) reader);
1781 }
1782 }
1783
1784
1785
1786
1787 abstract static class Comparators {
1788
1789
1790
1791
1792
1793
1794 static final Comparator<StoreFile> FLUSH_TIME =
1795 Ordering.compound(ImmutableList.of(
1796 Ordering.natural().onResultOf(new GetBulkTime()),
1797 Ordering.natural().onResultOf(new GetSeqId()),
1798 Ordering.natural().onResultOf(new GetPathName())
1799 ));
1800
1801 private static class GetBulkTime implements Function<StoreFile, Long> {
1802 @Override
1803 public Long apply(StoreFile sf) {
1804 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1805 return sf.getBulkLoadTimestamp();
1806 }
1807 }
1808 private static class GetSeqId implements Function<StoreFile, Long> {
1809 @Override
1810 public Long apply(StoreFile sf) {
1811 if (sf.isBulkLoadResult()) return -1L;
1812 return sf.getMaxSequenceId();
1813 }
1814 }
1815 private static class GetPathName implements Function<StoreFile, String> {
1816 @Override
1817 public String apply(StoreFile sf) {
1818 return sf.getPath().getName();
1819 }
1820 }
1821
1822
1823
1824
1825 static final Comparator<StoreFile> FILE_SIZE =
1826 Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1827 @Override
1828 public Long apply(StoreFile sf) {
1829 return sf.getReader().length();
1830 }
1831 });
1832 }
1833 }