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