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