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