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