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 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 HBaseFileSystem.deleteDirFromFileSystem(fs, getPath());
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 (!HBaseFileSystem.renameDirForFileSystem(fs, 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 private boolean includeMVCCReadpoint = true;
745
746 public WriterBuilder(Configuration conf, CacheConfig cacheConf,
747 FileSystem fs, int blockSize) {
748 this.conf = conf;
749 this.cacheConf = cacheConf;
750 this.fs = fs;
751 this.blockSize = blockSize;
752 }
753
754
755
756
757
758
759
760
761 public WriterBuilder withOutputDir(Path dir) {
762 Preconditions.checkNotNull(dir);
763 this.dir = dir;
764 return this;
765 }
766
767
768
769
770
771
772 public WriterBuilder withFilePath(Path filePath) {
773 Preconditions.checkNotNull(filePath);
774 this.filePath = filePath;
775 return this;
776 }
777
778 public WriterBuilder withCompression(Compression.Algorithm compressAlgo) {
779 Preconditions.checkNotNull(compressAlgo);
780 this.compressAlgo = compressAlgo;
781 return this;
782 }
783
784 public WriterBuilder withDataBlockEncoder(HFileDataBlockEncoder encoder) {
785 Preconditions.checkNotNull(encoder);
786 this.dataBlockEncoder = encoder;
787 return this;
788 }
789
790 public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
791 Preconditions.checkNotNull(comparator);
792 this.comparator = comparator;
793 return this;
794 }
795
796 public WriterBuilder withBloomType(BloomType bloomType) {
797 Preconditions.checkNotNull(bloomType);
798 this.bloomType = bloomType;
799 return this;
800 }
801
802
803
804
805
806 public WriterBuilder withMaxKeyCount(long maxKeyCount) {
807 this.maxKeyCount = maxKeyCount;
808 return this;
809 }
810
811
812
813
814
815 public WriterBuilder withChecksumType(ChecksumType checksumType) {
816 this.checksumType = checksumType;
817 return this;
818 }
819
820
821
822
823
824 public WriterBuilder withBytesPerChecksum(int bytesPerChecksum) {
825 this.bytesPerChecksum = bytesPerChecksum;
826 return this;
827 }
828
829
830
831
832
833 public WriterBuilder includeMVCCReadpoint(boolean includeMVCCReadpoint) {
834 this.includeMVCCReadpoint = includeMVCCReadpoint;
835 return this;
836 }
837
838
839
840
841
842
843 public Writer build() throws IOException {
844 if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
845 throw new IllegalArgumentException("Either specify parent directory " +
846 "or file path");
847 }
848
849 if (dir == null) {
850 dir = filePath.getParent();
851 }
852
853 if (!fs.exists(dir)) {
854 HBaseFileSystem.makeDirOnFileSystem(fs, dir);
855 }
856
857 if (filePath == null) {
858 filePath = getUniqueFile(fs, dir);
859 if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
860 bloomType = BloomType.NONE;
861 }
862 }
863
864 if (compressAlgo == null) {
865 compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
866 }
867 if (comparator == null) {
868 comparator = KeyValue.COMPARATOR;
869 }
870 return new Writer(fs, filePath, blockSize, compressAlgo, dataBlockEncoder,
871 conf, cacheConf, comparator, bloomType, maxKeyCount, checksumType,
872 bytesPerChecksum, includeMVCCReadpoint);
873 }
874 }
875
876
877
878
879
880
881 public static Path getUniqueFile(final FileSystem fs, final Path dir)
882 throws IOException {
883 if (!fs.getFileStatus(dir).isDir()) {
884 throw new IOException("Expecting " + dir.toString() +
885 " to be a directory");
886 }
887 return getRandomFilename(fs, dir);
888 }
889
890
891
892
893
894
895
896
897 static Path getRandomFilename(final FileSystem fs, final Path dir)
898 throws IOException {
899 return getRandomFilename(fs, dir, null);
900 }
901
902
903
904
905
906
907
908
909
910 static Path getRandomFilename(final FileSystem fs,
911 final Path dir,
912 final String suffix)
913 throws IOException {
914 return new Path(dir, UUID.randomUUID().toString().replaceAll("-", "")
915 + (suffix == null ? "" : suffix));
916 }
917
918
919
920
921
922
923 public static boolean validateStoreFileName(String fileName) {
924 if (HFileLink.isHFileLink(fileName))
925 return true;
926 if (isReference(fileName))
927 return true;
928 return !fileName.contains("-");
929 }
930
931
932
933
934
935
936
937
938
939
940
941
942
943 static Path split(final FileSystem fs,
944 final Path splitDir,
945 final StoreFile f,
946 final byte [] splitRow,
947 final Reference.Range range)
948 throws IOException {
949
950
951
952 if (range == Reference.Range.bottom) {
953
954 KeyValue splitKey = KeyValue.createLastOnRow(splitRow);
955 byte[] firstKey = f.createReader().getFirstKey();
956 if (f.getReader().getComparator().compare(splitKey.getBuffer(),
957 splitKey.getKeyOffset(), splitKey.getKeyLength(),
958 firstKey, 0, firstKey.length) < 0) {
959 return null;
960 }
961 }
962 else {
963
964 KeyValue splitKey = KeyValue.createFirstOnRow(splitRow);
965 byte[] lastKey = f.createReader().getLastKey();
966 if (f.getReader().getComparator().compare(splitKey.getBuffer(),
967 splitKey.getKeyOffset(), splitKey.getKeyLength(),
968 lastKey, 0, lastKey.length) > 0) {
969 return null;
970 }
971 }
972
973
974 Reference r = new Reference(splitRow, range);
975
976
977
978
979 String parentRegionName = f.getPath().getParent().getParent().getName();
980
981
982 Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
983 return r.write(fs, p);
984 }
985
986
987
988
989
990
991 public static class Writer {
992 private final BloomFilterWriter generalBloomFilterWriter;
993 private final BloomFilterWriter deleteFamilyBloomFilterWriter;
994 private final BloomType bloomType;
995 private byte[] lastBloomKey;
996 private int lastBloomKeyOffset, lastBloomKeyLen;
997 private KVComparator kvComparator;
998 private KeyValue lastKv = null;
999 private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
1000 private KeyValue lastDeleteFamilyKV = null;
1001 private long deleteFamilyCnt = 0;
1002
1003 protected HFileDataBlockEncoder dataBlockEncoder;
1004
1005
1006 protected ChecksumType checksumType;
1007
1008
1009 protected int bytesPerChecksum;
1010
1011 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
1012
1013
1014
1015
1016
1017
1018 boolean isTimeRangeTrackerSet = false;
1019
1020 protected HFile.Writer writer;
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038 private Writer(FileSystem fs, Path path, int blocksize,
1039 Compression.Algorithm compress,
1040 HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
1041 CacheConfig cacheConf,
1042 final KVComparator comparator, BloomType bloomType, long maxKeys,
1043 final ChecksumType checksumType, final int bytesPerChecksum, boolean includeMVCCReadpoint)
1044 throws IOException {
1045 this.dataBlockEncoder = dataBlockEncoder != null ?
1046 dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
1047 writer = HFile.getWriterFactory(conf, cacheConf)
1048 .withPath(fs, path)
1049 .withBlockSize(blocksize)
1050 .withCompression(compress)
1051 .withDataBlockEncoder(dataBlockEncoder)
1052 .withComparator(comparator.getRawComparator())
1053 .withChecksumType(checksumType)
1054 .withBytesPerChecksum(bytesPerChecksum)
1055 .includeMVCCReadpoint(includeMVCCReadpoint)
1056 .create();
1057
1058 this.kvComparator = comparator;
1059
1060 generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
1061 conf, cacheConf, bloomType,
1062 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1063
1064 if (generalBloomFilterWriter != null) {
1065 this.bloomType = bloomType;
1066 LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
1067 + generalBloomFilterWriter.getClass().getSimpleName());
1068 } else {
1069
1070 this.bloomType = BloomType.NONE;
1071 }
1072
1073
1074
1075 if (this.bloomType != BloomType.ROWCOL) {
1076 this.deleteFamilyBloomFilterWriter = BloomFilterFactory
1077 .createDeleteBloomAtWrite(conf, cacheConf,
1078 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
1079 } else {
1080 deleteFamilyBloomFilterWriter = null;
1081 }
1082 if (deleteFamilyBloomFilterWriter != null) {
1083 LOG.info("Delete Family Bloom filter type for " + path + ": "
1084 + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
1085 }
1086 this.checksumType = checksumType;
1087 this.bytesPerChecksum = bytesPerChecksum;
1088 }
1089
1090
1091
1092
1093
1094
1095
1096
1097 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
1098 throws IOException {
1099 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
1100 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
1101 Bytes.toBytes(majorCompaction));
1102 appendTrackedTimestampsToMetadata();
1103 }
1104
1105
1106
1107
1108 public void appendTrackedTimestampsToMetadata() throws IOException {
1109 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
1110 appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
1111 }
1112
1113
1114
1115
1116
1117 public void setTimeRangeTracker(final TimeRangeTracker trt) {
1118 this.timeRangeTracker = trt;
1119 isTimeRangeTrackerSet = true;
1120 }
1121
1122
1123
1124
1125
1126
1127
1128
1129 public void trackTimestamps(final KeyValue kv) {
1130 if (KeyValue.Type.Put.getCode() == kv.getType()) {
1131 earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
1132 }
1133 if (!isTimeRangeTrackerSet) {
1134 timeRangeTracker.includeTimestamp(kv);
1135 }
1136 }
1137
1138 private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
1139 if (this.generalBloomFilterWriter != null) {
1140
1141 boolean newKey = true;
1142 if (this.lastKv != null) {
1143 switch(bloomType) {
1144 case ROW:
1145 newKey = ! kvComparator.matchingRows(kv, lastKv);
1146 break;
1147 case ROWCOL:
1148 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
1149 break;
1150 case NONE:
1151 newKey = false;
1152 break;
1153 default:
1154 throw new IOException("Invalid Bloom filter type: " + bloomType +
1155 " (ROW or ROWCOL expected)");
1156 }
1157 }
1158 if (newKey) {
1159
1160
1161
1162
1163
1164
1165
1166
1167 byte[] bloomKey;
1168 int bloomKeyOffset, bloomKeyLen;
1169
1170 switch (bloomType) {
1171 case ROW:
1172 bloomKey = kv.getBuffer();
1173 bloomKeyOffset = kv.getRowOffset();
1174 bloomKeyLen = kv.getRowLength();
1175 break;
1176 case ROWCOL:
1177
1178
1179
1180 bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
1181 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
1182 kv.getQualifierOffset(), kv.getQualifierLength());
1183 bloomKeyOffset = 0;
1184 bloomKeyLen = bloomKey.length;
1185 break;
1186 default:
1187 throw new IOException("Invalid Bloom filter type: " + bloomType +
1188 " (ROW or ROWCOL expected)");
1189 }
1190 generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
1191 if (lastBloomKey != null
1192 && generalBloomFilterWriter.getComparator().compare(bloomKey,
1193 bloomKeyOffset, bloomKeyLen, lastBloomKey,
1194 lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
1195 throw new IOException("Non-increasing Bloom keys: "
1196 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
1197 + " after "
1198 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
1199 lastBloomKeyLen));
1200 }
1201 lastBloomKey = bloomKey;
1202 lastBloomKeyOffset = bloomKeyOffset;
1203 lastBloomKeyLen = bloomKeyLen;
1204 this.lastKv = kv;
1205 }
1206 }
1207 }
1208
1209 private void appendDeleteFamilyBloomFilter(final KeyValue kv)
1210 throws IOException {
1211 if (!kv.isDeleteFamily()) {
1212 return;
1213 }
1214
1215
1216 deleteFamilyCnt++;
1217 if (null != this.deleteFamilyBloomFilterWriter) {
1218 boolean newKey = true;
1219 if (lastDeleteFamilyKV != null) {
1220 newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
1221 }
1222 if (newKey) {
1223 this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
1224 kv.getRowOffset(), kv.getRowLength());
1225 this.lastDeleteFamilyKV = kv;
1226 }
1227 }
1228 }
1229
1230 public void append(final KeyValue kv) throws IOException {
1231 appendGeneralBloomfilter(kv);
1232 appendDeleteFamilyBloomFilter(kv);
1233 writer.append(kv);
1234 trackTimestamps(kv);
1235 }
1236
1237 public Path getPath() {
1238 return this.writer.getPath();
1239 }
1240
1241 boolean hasGeneralBloom() {
1242 return this.generalBloomFilterWriter != null;
1243 }
1244
1245
1246
1247
1248
1249
1250 BloomFilterWriter getGeneralBloomWriter() {
1251 return generalBloomFilterWriter;
1252 }
1253
1254 private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
1255 boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
1256 if (haveBloom) {
1257 bfw.compactBloom();
1258 }
1259 return haveBloom;
1260 }
1261
1262 private boolean closeGeneralBloomFilter() throws IOException {
1263 boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
1264
1265
1266 if (hasGeneralBloom) {
1267 writer.addGeneralBloomFilter(generalBloomFilterWriter);
1268 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
1269 Bytes.toBytes(bloomType.toString()));
1270 if (lastBloomKey != null) {
1271 writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1272 lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1273 + lastBloomKeyLen));
1274 }
1275 }
1276 return hasGeneralBloom;
1277 }
1278
1279 private boolean closeDeleteFamilyBloomFilter() throws IOException {
1280 boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1281
1282
1283 if (hasDeleteFamilyBloom) {
1284 writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1285 }
1286
1287
1288
1289 writer.appendFileInfo(DELETE_FAMILY_COUNT,
1290 Bytes.toBytes(this.deleteFamilyCnt));
1291
1292 return hasDeleteFamilyBloom;
1293 }
1294
1295 public void close() throws IOException {
1296 boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1297 boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1298
1299 writer.close();
1300
1301
1302
1303 StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
1304 + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
1305 + " was added to HFile (" + getPath() + ") ");
1306
1307 }
1308
1309 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1310 writer.appendFileInfo(key, value);
1311 }
1312
1313
1314 HFile.Writer getHFileWriter() {
1315 return writer;
1316 }
1317 }
1318
1319
1320
1321
1322 public static class Reader extends SchemaConfigured {
1323 static final Log LOG = LogFactory.getLog(Reader.class.getName());
1324
1325 protected BloomFilter generalBloomFilter = null;
1326 protected BloomFilter deleteFamilyBloomFilter = null;
1327 protected BloomType bloomFilterType;
1328 private final HFile.Reader reader;
1329 protected TimeRangeTracker timeRangeTracker = null;
1330 protected long sequenceID = -1;
1331 private byte[] lastBloomKey;
1332 private long deleteFamilyCnt = -1;
1333
1334 public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
1335 DataBlockEncoding preferredEncodingInCache) throws IOException {
1336 super(path);
1337 reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
1338 preferredEncodingInCache);
1339 bloomFilterType = BloomType.NONE;
1340 }
1341
1342 public Reader(FileSystem fs, Path path, HFileLink hfileLink, long size,
1343 CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
1344 boolean closeIStream) throws IOException {
1345 super(path);
1346
1347 FSDataInputStream in = hfileLink.open(fs);
1348 FSDataInputStream inNoChecksum = in;
1349 if (fs instanceof HFileSystem) {
1350 FileSystem noChecksumFs = ((HFileSystem)fs).getNoChecksumFs();
1351 inNoChecksum = hfileLink.open(noChecksumFs);
1352 }
1353
1354 reader = HFile.createReaderWithEncoding(fs, path, in, inNoChecksum,
1355 size, cacheConf, preferredEncodingInCache, closeIStream);
1356 bloomFilterType = BloomType.NONE;
1357 }
1358
1359
1360
1361
1362 Reader() {
1363 this.reader = null;
1364 }
1365
1366 public RawComparator<byte []> getComparator() {
1367 return reader.getComparator();
1368 }
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1379 boolean pread) {
1380 return getStoreFileScanner(cacheBlocks, pread, false);
1381 }
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1392 boolean pread,
1393 boolean isCompaction) {
1394 return new StoreFileScanner(this,
1395 getScanner(cacheBlocks, pread,
1396 isCompaction), !isCompaction);
1397 }
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408 @Deprecated
1409 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1410 return getScanner(cacheBlocks, pread, false);
1411 }
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426 @Deprecated
1427 public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1428 boolean isCompaction) {
1429 return reader.getScanner(cacheBlocks, pread, isCompaction);
1430 }
1431
1432 public void close(boolean evictOnClose) throws IOException {
1433 reader.close(evictOnClose);
1434 }
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444 boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1445 if (timeRangeTracker == null) {
1446 return true;
1447 } else {
1448 return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1449 timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1450 }
1451 }
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469 boolean passesBloomFilter(Scan scan,
1470 final SortedSet<byte[]> columns) {
1471
1472
1473 if (!scan.isGetScan()) {
1474 return true;
1475 }
1476
1477 byte[] row = scan.getStartRow();
1478 switch (this.bloomFilterType) {
1479 case ROW:
1480 return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1481
1482 case ROWCOL:
1483 if (columns != null && columns.size() == 1) {
1484 byte[] column = columns.first();
1485 return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1486 column.length);
1487 }
1488
1489
1490
1491 return true;
1492
1493 default:
1494 return true;
1495 }
1496 }
1497
1498 public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1499 int rowLen) {
1500
1501
1502 BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1503
1504
1505 if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1506 return false;
1507 }
1508
1509 if (bloomFilter == null) {
1510 return true;
1511 }
1512
1513 try {
1514 if (!bloomFilter.supportsAutoLoading()) {
1515 return true;
1516 }
1517 return bloomFilter.contains(row, rowOffset, rowLen, null);
1518 } catch (IllegalArgumentException e) {
1519 LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1520 e);
1521 setDeleteFamilyBloomFilterFaulty();
1522 }
1523
1524 return true;
1525 }
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539 public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1540 int rowLen, byte[] col, int colOffset, int colLen) {
1541 if (generalBloomFilter == null)
1542 return true;
1543
1544 byte[] key;
1545 switch (bloomFilterType) {
1546 case ROW:
1547 if (col != null) {
1548 throw new RuntimeException("Row-only Bloom filter called with " +
1549 "column specified");
1550 }
1551 if (rowOffset != 0 || rowLen != row.length) {
1552 throw new AssertionError("For row-only Bloom filters the row "
1553 + "must occupy the whole array");
1554 }
1555 key = row;
1556 break;
1557
1558 case ROWCOL:
1559 key = generalBloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1560 colOffset, colLen);
1561 break;
1562
1563 default:
1564 return true;
1565 }
1566
1567
1568
1569 BloomFilter bloomFilter = this.generalBloomFilter;
1570
1571 if (bloomFilter == null) {
1572 return true;
1573 }
1574
1575
1576 if (reader.getTrailer().getEntryCount() == 0)
1577 return false;
1578
1579 try {
1580 boolean shouldCheckBloom;
1581 ByteBuffer bloom;
1582 if (bloomFilter.supportsAutoLoading()) {
1583 bloom = null;
1584 shouldCheckBloom = true;
1585 } else {
1586 bloom = reader.getMetaBlock(HFileWriterV1.BLOOM_FILTER_DATA_KEY,
1587 true);
1588 shouldCheckBloom = bloom != null;
1589 }
1590
1591 if (shouldCheckBloom) {
1592 boolean exists;
1593
1594
1595
1596
1597 boolean keyIsAfterLast = lastBloomKey != null
1598 && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1599
1600 if (bloomFilterType == BloomType.ROWCOL) {
1601
1602
1603
1604
1605 byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1606 null, 0, 0);
1607
1608 if (keyIsAfterLast
1609 && bloomFilter.getComparator().compare(rowBloomKey,
1610 lastBloomKey) > 0) {
1611 exists = false;
1612 } else {
1613 exists =
1614 bloomFilter.contains(key, 0, key.length, bloom) ||
1615 bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1616 bloom);
1617 }
1618 } else {
1619 exists = !keyIsAfterLast
1620 && bloomFilter.contains(key, 0, key.length, bloom);
1621 }
1622
1623 getSchemaMetrics().updateBloomMetrics(exists);
1624 return exists;
1625 }
1626 } catch (IOException e) {
1627 LOG.error("Error reading bloom filter data -- proceeding without",
1628 e);
1629 setGeneralBloomFilterFaulty();
1630 } catch (IllegalArgumentException e) {
1631 LOG.error("Bad bloom filter data -- proceeding without", e);
1632 setGeneralBloomFilterFaulty();
1633 }
1634
1635 return true;
1636 }
1637
1638
1639
1640
1641
1642
1643 public boolean passesKeyRangeFilter(Scan scan) {
1644 if (this.getFirstKey() == null || this.getLastKey() == null) {
1645
1646 return false;
1647 }
1648 if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
1649 && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
1650 return true;
1651 }
1652 KeyValue startKeyValue = KeyValue.createFirstOnRow(scan.getStartRow());
1653 KeyValue stopKeyValue = KeyValue.createLastOnRow(scan.getStopRow());
1654 boolean nonOverLapping = (getComparator().compare(this.getFirstKey(),
1655 stopKeyValue.getKey()) > 0 && !Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW))
1656 || getComparator().compare(this.getLastKey(), startKeyValue.getKey()) < 0;
1657 return !nonOverLapping;
1658 }
1659
1660 public Map<byte[], byte[]> loadFileInfo() throws IOException {
1661 Map<byte [], byte []> fi = reader.loadFileInfo();
1662
1663 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1664 if (b != null) {
1665 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1666 }
1667
1668 lastBloomKey = fi.get(LAST_BLOOM_KEY);
1669 byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1670 if (cnt != null) {
1671 deleteFamilyCnt = Bytes.toLong(cnt);
1672 }
1673
1674 return fi;
1675 }
1676
1677 public void loadBloomfilter() {
1678 this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1679 this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1680 }
1681
1682 private void loadBloomfilter(BlockType blockType) {
1683 try {
1684 if (blockType == BlockType.GENERAL_BLOOM_META) {
1685 if (this.generalBloomFilter != null)
1686 return;
1687
1688 DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1689 if (bloomMeta != null) {
1690
1691 if (bloomFilterType == BloomType.NONE) {
1692 throw new IOException(
1693 "valid bloom filter type not found in FileInfo");
1694 } else {
1695 generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1696 reader);
1697 LOG.info("Loaded " + bloomFilterType.toString() + " ("
1698 + generalBloomFilter.getClass().getSimpleName()
1699 + ") metadata for " + reader.getName());
1700 }
1701 }
1702 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1703 if (this.deleteFamilyBloomFilter != null)
1704 return;
1705
1706 DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1707 if (bloomMeta != null) {
1708 deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1709 bloomMeta, reader);
1710 LOG.info("Loaded Delete Family Bloom ("
1711 + deleteFamilyBloomFilter.getClass().getSimpleName()
1712 + ") metadata for " + reader.getName());
1713 }
1714 } else {
1715 throw new RuntimeException("Block Type: " + blockType.toString()
1716 + "is not supported for Bloom filter");
1717 }
1718 } catch (IOException e) {
1719 LOG.error("Error reading bloom filter meta for " + blockType
1720 + " -- proceeding without", e);
1721 setBloomFilterFaulty(blockType);
1722 } catch (IllegalArgumentException e) {
1723 LOG.error("Bad bloom filter meta " + blockType
1724 + " -- proceeding without", e);
1725 setBloomFilterFaulty(blockType);
1726 }
1727 }
1728
1729 private void setBloomFilterFaulty(BlockType blockType) {
1730 if (blockType == BlockType.GENERAL_BLOOM_META) {
1731 setGeneralBloomFilterFaulty();
1732 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1733 setDeleteFamilyBloomFilterFaulty();
1734 }
1735 }
1736
1737
1738
1739
1740
1741
1742
1743
1744 public long getFilterEntries() {
1745 return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1746 : reader.getEntries();
1747 }
1748
1749 public void setGeneralBloomFilterFaulty() {
1750 generalBloomFilter = null;
1751 }
1752
1753 public void setDeleteFamilyBloomFilterFaulty() {
1754 this.deleteFamilyBloomFilter = null;
1755 }
1756
1757 public byte[] getLastKey() {
1758 return reader.getLastKey();
1759 }
1760
1761 public byte[] midkey() throws IOException {
1762 return reader.midkey();
1763 }
1764
1765 public long length() {
1766 return reader.length();
1767 }
1768
1769 public long getTotalUncompressedBytes() {
1770 return reader.getTrailer().getTotalUncompressedBytes();
1771 }
1772
1773 public long getEntries() {
1774 return reader.getEntries();
1775 }
1776
1777 public long getDeleteFamilyCnt() {
1778 return deleteFamilyCnt;
1779 }
1780
1781 public byte[] getFirstKey() {
1782 return reader.getFirstKey();
1783 }
1784
1785 public long indexSize() {
1786 return reader.indexSize();
1787 }
1788
1789 public String getColumnFamilyName() {
1790 return reader.getColumnFamilyName();
1791 }
1792
1793 public BloomType getBloomFilterType() {
1794 return this.bloomFilterType;
1795 }
1796
1797 public long getSequenceID() {
1798 return sequenceID;
1799 }
1800
1801 public void setSequenceID(long sequenceID) {
1802 this.sequenceID = sequenceID;
1803 }
1804
1805 BloomFilter getGeneralBloomFilter() {
1806 return generalBloomFilter;
1807 }
1808
1809 long getUncompressedDataIndexSize() {
1810 return reader.getTrailer().getUncompressedDataIndexSize();
1811 }
1812
1813 public long getTotalBloomSize() {
1814 if (generalBloomFilter == null)
1815 return 0;
1816 return generalBloomFilter.getByteSize();
1817 }
1818
1819 public int getHFileVersion() {
1820 return reader.getTrailer().getMajorVersion();
1821 }
1822
1823 HFile.Reader getHFileReader() {
1824 return reader;
1825 }
1826
1827 void disableBloomFilterForTesting() {
1828 generalBloomFilter = null;
1829 this.deleteFamilyBloomFilter = null;
1830 }
1831
1832 public long getMaxTimestamp() {
1833 return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.maximumTimestamp;
1834 }
1835
1836 @Override
1837 public void schemaConfigurationChanged() {
1838 passSchemaMetricsTo((SchemaConfigured) reader);
1839 }
1840 }
1841
1842
1843
1844
1845 abstract static class Comparators {
1846
1847
1848
1849
1850
1851
1852 static final Comparator<StoreFile> FLUSH_TIME =
1853 Ordering.compound(ImmutableList.of(
1854 Ordering.natural().onResultOf(new GetBulkTime()),
1855 Ordering.natural().onResultOf(new GetSeqId()),
1856 Ordering.natural().onResultOf(new GetPathName())
1857 ));
1858
1859 private static class GetBulkTime implements Function<StoreFile, Long> {
1860 @Override
1861 public Long apply(StoreFile sf) {
1862 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1863 return sf.getBulkLoadTimestamp();
1864 }
1865 }
1866 private static class GetSeqId implements Function<StoreFile, Long> {
1867 @Override
1868 public Long apply(StoreFile sf) {
1869 if (sf.isBulkLoadResult()) return -1L;
1870 return sf.getMaxSequenceId();
1871 }
1872 }
1873 private static class GetPathName implements Function<StoreFile, String> {
1874 @Override
1875 public String apply(StoreFile sf) {
1876 return sf.getPath().getName();
1877 }
1878 }
1879
1880
1881
1882
1883 static final Comparator<StoreFile> FILE_SIZE =
1884 Ordering.natural().reverse().onResultOf(new Function<StoreFile, Long>() {
1885 @Override
1886 public Long apply(StoreFile sf) {
1887 return sf.getReader().length();
1888 }
1889 });
1890 }
1891 }