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