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.FileNotFoundException;
23 import java.io.IOException;
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.FSDataInputStream;
39 import org.apache.hadoop.fs.FileSystem;
40 import org.apache.hadoop.fs.Path;
41 import org.apache.hadoop.hbase.HConstants;
42 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
43 import org.apache.hadoop.hbase.KeyValue;
44 import org.apache.hadoop.hbase.KeyValue.KVComparator;
45 import org.apache.hadoop.hbase.client.Scan;
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 ChecksumType checksumType = HFile.DEFAULT_CHECKSUM_TYPE;
531 private int bytesPerChecksum = HFile.DEFAULT_BYTES_PER_CHECKSUM;
532 private boolean includeMVCCReadpoint = true;
533
534 public WriterBuilder(Configuration conf, CacheConfig cacheConf,
535 FileSystem fs, int blockSize) {
536 this.conf = conf;
537 this.cacheConf = cacheConf;
538 this.fs = fs;
539 this.blockSize = blockSize;
540 }
541
542
543
544
545
546
547
548
549 public WriterBuilder withOutputDir(Path dir) {
550 Preconditions.checkNotNull(dir);
551 this.dir = dir;
552 return this;
553 }
554
555
556
557
558
559
560 public WriterBuilder withFilePath(Path filePath) {
561 Preconditions.checkNotNull(filePath);
562 this.filePath = filePath;
563 return this;
564 }
565
566 public WriterBuilder withCompression(Compression.Algorithm compressAlgo) {
567 Preconditions.checkNotNull(compressAlgo);
568 this.compressAlgo = compressAlgo;
569 return this;
570 }
571
572 public WriterBuilder withDataBlockEncoder(HFileDataBlockEncoder encoder) {
573 Preconditions.checkNotNull(encoder);
574 this.dataBlockEncoder = encoder;
575 return this;
576 }
577
578 public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
579 Preconditions.checkNotNull(comparator);
580 this.comparator = comparator;
581 return this;
582 }
583
584 public WriterBuilder withBloomType(BloomType bloomType) {
585 Preconditions.checkNotNull(bloomType);
586 this.bloomType = bloomType;
587 return this;
588 }
589
590
591
592
593
594 public WriterBuilder withMaxKeyCount(long maxKeyCount) {
595 this.maxKeyCount = maxKeyCount;
596 return this;
597 }
598
599
600
601
602
603 public WriterBuilder withChecksumType(ChecksumType checksumType) {
604 this.checksumType = checksumType;
605 return this;
606 }
607
608
609
610
611
612 public WriterBuilder withBytesPerChecksum(int bytesPerChecksum) {
613 this.bytesPerChecksum = bytesPerChecksum;
614 return this;
615 }
616
617
618
619
620
621 public WriterBuilder includeMVCCReadpoint(boolean includeMVCCReadpoint) {
622 this.includeMVCCReadpoint = includeMVCCReadpoint;
623 return this;
624 }
625
626
627
628
629
630
631 public Writer build() throws IOException {
632 if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
633 throw new IllegalArgumentException("Either specify parent directory " +
634 "or file path");
635 }
636
637 if (dir == null) {
638 dir = filePath.getParent();
639 }
640
641 if (!fs.exists(dir)) {
642 fs.mkdirs(dir);
643 }
644
645 if (filePath == null) {
646 filePath = getUniqueFile(fs, dir);
647 if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
648 bloomType = BloomType.NONE;
649 }
650 }
651
652 if (compressAlgo == null) {
653 compressAlgo = HFile.DEFAULT_COMPRESSION_ALGORITHM;
654 }
655 if (comparator == null) {
656 comparator = KeyValue.COMPARATOR;
657 }
658 return new Writer(fs, filePath, blockSize, compressAlgo, dataBlockEncoder,
659 conf, cacheConf, comparator, bloomType, maxKeyCount, checksumType,
660 bytesPerChecksum, includeMVCCReadpoint);
661 }
662 }
663
664
665
666
667
668
669 public static Path getUniqueFile(final FileSystem fs, final Path dir)
670 throws IOException {
671 if (!fs.getFileStatus(dir).isDir()) {
672 throw new IOException("Expecting " + dir.toString() +
673 " to be a directory");
674 }
675 return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
676 }
677
678 public Long getMinimumTimestamp() {
679 return (getReader().timeRangeTracker == null) ?
680 null :
681 getReader().timeRangeTracker.minimumTimestamp;
682 }
683
684
685
686
687
688
689 byte[] getFileSplitPoint(KVComparator comparator) throws IOException {
690 if (this.reader == null) {
691 LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
692 return null;
693 }
694
695
696
697 byte [] midkey = this.reader.midkey();
698 if (midkey != null) {
699 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
700 byte [] fk = this.reader.getFirstKey();
701 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
702 byte [] lk = this.reader.getLastKey();
703 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
704
705 if (comparator.compareRows(mk, firstKey) == 0 || comparator.compareRows(mk, lastKey) == 0) {
706 if (LOG.isDebugEnabled()) {
707 LOG.debug("cannot split because midkey is the same as first or last row");
708 }
709 return null;
710 }
711 return mk.getRow();
712 }
713 return null;
714 }
715
716
717
718
719
720 public static class Writer implements Compactor.CellSink {
721 private final BloomFilterWriter generalBloomFilterWriter;
722 private final BloomFilterWriter deleteFamilyBloomFilterWriter;
723 private final BloomType bloomType;
724 private byte[] lastBloomKey;
725 private int lastBloomKeyOffset, lastBloomKeyLen;
726 private KVComparator kvComparator;
727 private KeyValue lastKv = null;
728 private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
729 private KeyValue lastDeleteFamilyKV = null;
730 private long deleteFamilyCnt = 0;
731
732 protected HFileDataBlockEncoder dataBlockEncoder;
733
734
735 protected ChecksumType checksumType;
736
737
738 protected int bytesPerChecksum;
739
740 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
741
742
743
744
745
746
747 boolean isTimeRangeTrackerSet = false;
748
749 protected HFile.Writer writer;
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767 private Writer(FileSystem fs, Path path, int blocksize,
768 Compression.Algorithm compress,
769 HFileDataBlockEncoder dataBlockEncoder, final Configuration conf,
770 CacheConfig cacheConf,
771 final KVComparator comparator, BloomType bloomType, long maxKeys,
772 final ChecksumType checksumType, final int bytesPerChecksum,
773 final boolean includeMVCCReadpoint) throws IOException {
774 this.dataBlockEncoder = dataBlockEncoder != null ?
775 dataBlockEncoder : NoOpDataBlockEncoder.INSTANCE;
776 writer = HFile.getWriterFactory(conf, cacheConf)
777 .withPath(fs, path)
778 .withBlockSize(blocksize)
779 .withCompression(compress)
780 .withDataBlockEncoder(this.dataBlockEncoder)
781 .withComparator(comparator.getRawComparator())
782 .withChecksumType(checksumType)
783 .withBytesPerChecksum(bytesPerChecksum)
784 .includeMVCCReadpoint(includeMVCCReadpoint)
785 .create();
786
787 this.kvComparator = comparator;
788
789 generalBloomFilterWriter = BloomFilterFactory.createGeneralBloomAtWrite(
790 conf, cacheConf, bloomType,
791 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
792
793 if (generalBloomFilterWriter != null) {
794 this.bloomType = bloomType;
795 LOG.info("Bloom filter type for " + path + ": " + this.bloomType + ", "
796 + generalBloomFilterWriter.getClass().getSimpleName());
797 } else {
798
799 this.bloomType = BloomType.NONE;
800 }
801
802
803
804 if (this.bloomType != BloomType.ROWCOL) {
805 this.deleteFamilyBloomFilterWriter = BloomFilterFactory
806 .createDeleteBloomAtWrite(conf, cacheConf,
807 (int) Math.min(maxKeys, Integer.MAX_VALUE), writer);
808 } else {
809 deleteFamilyBloomFilterWriter = null;
810 }
811 if (deleteFamilyBloomFilterWriter != null) {
812 LOG.info("Delete Family Bloom filter type for " + path + ": "
813 + deleteFamilyBloomFilterWriter.getClass().getSimpleName());
814 }
815 this.checksumType = checksumType;
816 this.bytesPerChecksum = bytesPerChecksum;
817 }
818
819
820
821
822
823
824
825
826 public void appendMetadata(final long maxSequenceId, final boolean majorCompaction)
827 throws IOException {
828 writer.appendFileInfo(MAX_SEQ_ID_KEY, Bytes.toBytes(maxSequenceId));
829 writer.appendFileInfo(MAJOR_COMPACTION_KEY,
830 Bytes.toBytes(majorCompaction));
831 appendTrackedTimestampsToMetadata();
832 }
833
834
835
836
837 public void appendTrackedTimestampsToMetadata() throws IOException {
838 appendFileInfo(TIMERANGE_KEY,WritableUtils.toByteArray(timeRangeTracker));
839 appendFileInfo(EARLIEST_PUT_TS, Bytes.toBytes(earliestPutTs));
840 }
841
842
843
844
845
846 public void setTimeRangeTracker(final TimeRangeTracker trt) {
847 this.timeRangeTracker = trt;
848 isTimeRangeTrackerSet = true;
849 }
850
851
852
853
854
855
856
857
858 public void trackTimestamps(final KeyValue kv) {
859 if (KeyValue.Type.Put.getCode() == kv.getType()) {
860 earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
861 }
862 if (!isTimeRangeTrackerSet) {
863 timeRangeTracker.includeTimestamp(kv);
864 }
865 }
866
867 private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
868 if (this.generalBloomFilterWriter != null) {
869
870 boolean newKey = true;
871 if (this.lastKv != null) {
872 switch(bloomType) {
873 case ROW:
874 newKey = ! kvComparator.matchingRows(kv, lastKv);
875 break;
876 case ROWCOL:
877 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
878 break;
879 case NONE:
880 newKey = false;
881 break;
882 default:
883 throw new IOException("Invalid Bloom filter type: " + bloomType +
884 " (ROW or ROWCOL expected)");
885 }
886 }
887 if (newKey) {
888
889
890
891
892
893
894
895
896 byte[] bloomKey;
897 int bloomKeyOffset, bloomKeyLen;
898
899 switch (bloomType) {
900 case ROW:
901 bloomKey = kv.getBuffer();
902 bloomKeyOffset = kv.getRowOffset();
903 bloomKeyLen = kv.getRowLength();
904 break;
905 case ROWCOL:
906
907
908
909 bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
910 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
911 kv.getQualifierOffset(), kv.getQualifierLength());
912 bloomKeyOffset = 0;
913 bloomKeyLen = bloomKey.length;
914 break;
915 default:
916 throw new IOException("Invalid Bloom filter type: " + bloomType +
917 " (ROW or ROWCOL expected)");
918 }
919 generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
920 if (lastBloomKey != null
921 && generalBloomFilterWriter.getComparator().compare(bloomKey,
922 bloomKeyOffset, bloomKeyLen, lastBloomKey,
923 lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
924 throw new IOException("Non-increasing Bloom keys: "
925 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
926 + " after "
927 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
928 lastBloomKeyLen));
929 }
930 lastBloomKey = bloomKey;
931 lastBloomKeyOffset = bloomKeyOffset;
932 lastBloomKeyLen = bloomKeyLen;
933 this.lastKv = kv;
934 }
935 }
936 }
937
938 private void appendDeleteFamilyBloomFilter(final KeyValue kv)
939 throws IOException {
940 if (!kv.isDeleteFamily()) {
941 return;
942 }
943
944
945 deleteFamilyCnt++;
946 if (null != this.deleteFamilyBloomFilterWriter) {
947 boolean newKey = true;
948 if (lastDeleteFamilyKV != null) {
949 newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
950 }
951 if (newKey) {
952 this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
953 kv.getRowOffset(), kv.getRowLength());
954 this.lastDeleteFamilyKV = kv;
955 }
956 }
957 }
958
959 public void append(final KeyValue kv) throws IOException {
960 appendGeneralBloomfilter(kv);
961 appendDeleteFamilyBloomFilter(kv);
962 writer.append(kv);
963 trackTimestamps(kv);
964 }
965
966 public Path getPath() {
967 return this.writer.getPath();
968 }
969
970 boolean hasGeneralBloom() {
971 return this.generalBloomFilterWriter != null;
972 }
973
974
975
976
977
978
979 BloomFilterWriter getGeneralBloomWriter() {
980 return generalBloomFilterWriter;
981 }
982
983 private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
984 boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
985 if (haveBloom) {
986 bfw.compactBloom();
987 }
988 return haveBloom;
989 }
990
991 private boolean closeGeneralBloomFilter() throws IOException {
992 boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
993
994
995 if (hasGeneralBloom) {
996 writer.addGeneralBloomFilter(generalBloomFilterWriter);
997 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
998 Bytes.toBytes(bloomType.toString()));
999 if (lastBloomKey != null) {
1000 writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1001 lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1002 + lastBloomKeyLen));
1003 }
1004 }
1005 return hasGeneralBloom;
1006 }
1007
1008 private boolean closeDeleteFamilyBloomFilter() throws IOException {
1009 boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1010
1011
1012 if (hasDeleteFamilyBloom) {
1013 writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1014 }
1015
1016
1017
1018 writer.appendFileInfo(DELETE_FAMILY_COUNT,
1019 Bytes.toBytes(this.deleteFamilyCnt));
1020
1021 return hasDeleteFamilyBloom;
1022 }
1023
1024 public void close() throws IOException {
1025 boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1026 boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1027
1028 writer.close();
1029
1030
1031
1032 StoreFile.LOG.info((hasGeneralBloom ? "" : "NO ") + "General Bloom and "
1033 + (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily"
1034 + " was added to HFile (" + getPath() + ") ");
1035
1036 }
1037
1038 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1039 writer.appendFileInfo(key, value);
1040 }
1041
1042
1043
1044 HFile.Writer getHFileWriter() {
1045 return writer;
1046 }
1047 }
1048
1049
1050
1051
1052 public static class Reader {
1053 static final Log LOG = LogFactory.getLog(Reader.class.getName());
1054
1055 protected BloomFilter generalBloomFilter = null;
1056 protected BloomFilter deleteFamilyBloomFilter = null;
1057 protected BloomType bloomFilterType;
1058 private final HFile.Reader reader;
1059 protected TimeRangeTracker timeRangeTracker = null;
1060 protected long sequenceID = -1;
1061 private byte[] lastBloomKey;
1062 private long deleteFamilyCnt = -1;
1063
1064 public Reader(FileSystem fs, Path path, CacheConfig cacheConf,
1065 DataBlockEncoding preferredEncodingInCache) throws IOException {
1066 reader = HFile.createReaderWithEncoding(fs, path, cacheConf,
1067 preferredEncodingInCache);
1068 bloomFilterType = BloomType.NONE;
1069 }
1070
1071 public Reader(FileSystem fs, Path path, FSDataInputStream in,
1072 final FSDataInputStream inNoChecksum, long size,
1073 CacheConfig cacheConf, DataBlockEncoding preferredEncodingInCache,
1074 boolean closeIStream) throws IOException {
1075 reader = HFile.createReaderWithEncoding(fs, path, in, inNoChecksum,
1076 size, cacheConf, preferredEncodingInCache, closeIStream);
1077 bloomFilterType = BloomType.NONE;
1078 }
1079
1080
1081
1082
1083 Reader() {
1084 this.reader = null;
1085 }
1086
1087 public RawComparator<byte []> getComparator() {
1088 return reader.getComparator();
1089 }
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1100 boolean pread) {
1101 return getStoreFileScanner(cacheBlocks, pread, false);
1102 }
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1113 boolean pread,
1114 boolean isCompaction) {
1115 return new StoreFileScanner(this,
1116 getScanner(cacheBlocks, pread,
1117 isCompaction), !isCompaction);
1118 }
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129 @Deprecated
1130 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1131 return getScanner(cacheBlocks, pread, false);
1132 }
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147 @Deprecated
1148 public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1149 boolean isCompaction) {
1150 return reader.getScanner(cacheBlocks, pread, isCompaction);
1151 }
1152
1153 public void close(boolean evictOnClose) throws IOException {
1154 reader.close(evictOnClose);
1155 }
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165 boolean passesTimerangeFilter(Scan scan, long oldestUnexpiredTS) {
1166 if (timeRangeTracker == null) {
1167 return true;
1168 } else {
1169 return timeRangeTracker.includesTimeRange(scan.getTimeRange()) &&
1170 timeRangeTracker.getMaximumTimestamp() >= oldestUnexpiredTS;
1171 }
1172 }
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190 boolean passesBloomFilter(Scan scan,
1191 final SortedSet<byte[]> columns) {
1192
1193
1194 if (!scan.isGetScan()) {
1195 return true;
1196 }
1197
1198 byte[] row = scan.getStartRow();
1199 switch (this.bloomFilterType) {
1200 case ROW:
1201 return passesGeneralBloomFilter(row, 0, row.length, null, 0, 0);
1202
1203 case ROWCOL:
1204 if (columns != null && columns.size() == 1) {
1205 byte[] column = columns.first();
1206 return passesGeneralBloomFilter(row, 0, row.length, column, 0,
1207 column.length);
1208 }
1209
1210
1211
1212 return true;
1213
1214 default:
1215 return true;
1216 }
1217 }
1218
1219 public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
1220 int rowLen) {
1221
1222
1223 BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
1224
1225
1226 if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
1227 return false;
1228 }
1229
1230 if (bloomFilter == null) {
1231 return true;
1232 }
1233
1234 try {
1235 if (!bloomFilter.supportsAutoLoading()) {
1236 return true;
1237 }
1238 return bloomFilter.contains(row, rowOffset, rowLen, null);
1239 } catch (IllegalArgumentException e) {
1240 LOG.error("Bad Delete Family bloom filter data -- proceeding without",
1241 e);
1242 setDeleteFamilyBloomFilterFaulty();
1243 }
1244
1245 return true;
1246 }
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260 public boolean passesGeneralBloomFilter(byte[] row, int rowOffset,
1261 int rowLen, byte[] col, int colOffset, int colLen) {
1262
1263
1264 BloomFilter bloomFilter = this.generalBloomFilter;
1265 if (bloomFilter == null) {
1266 return true;
1267 }
1268
1269 byte[] key;
1270 switch (bloomFilterType) {
1271 case ROW:
1272 if (col != null) {
1273 throw new RuntimeException("Row-only Bloom filter called with " +
1274 "column specified");
1275 }
1276 if (rowOffset != 0 || rowLen != row.length) {
1277 throw new AssertionError("For row-only Bloom filters the row "
1278 + "must occupy the whole array");
1279 }
1280 key = row;
1281 break;
1282
1283 case ROWCOL:
1284 key = bloomFilter.createBloomKey(row, rowOffset, rowLen, col,
1285 colOffset, colLen);
1286 break;
1287
1288 default:
1289 return true;
1290 }
1291
1292
1293 if (reader.getTrailer().getEntryCount() == 0)
1294 return false;
1295
1296 try {
1297 boolean shouldCheckBloom;
1298 ByteBuffer bloom;
1299 if (bloomFilter.supportsAutoLoading()) {
1300 bloom = null;
1301 shouldCheckBloom = true;
1302 } else {
1303 bloom = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY,
1304 true);
1305 shouldCheckBloom = bloom != null;
1306 }
1307
1308 if (shouldCheckBloom) {
1309 boolean exists;
1310
1311
1312
1313
1314 boolean keyIsAfterLast = lastBloomKey != null
1315 && bloomFilter.getComparator().compare(key, lastBloomKey) > 0;
1316
1317 if (bloomFilterType == BloomType.ROWCOL) {
1318
1319
1320
1321
1322 byte[] rowBloomKey = bloomFilter.createBloomKey(row, 0, row.length,
1323 null, 0, 0);
1324
1325 if (keyIsAfterLast
1326 && bloomFilter.getComparator().compare(rowBloomKey,
1327 lastBloomKey) > 0) {
1328 exists = false;
1329 } else {
1330 exists =
1331 bloomFilter.contains(key, 0, key.length, bloom) ||
1332 bloomFilter.contains(rowBloomKey, 0, rowBloomKey.length,
1333 bloom);
1334 }
1335 } else {
1336 exists = !keyIsAfterLast
1337 && bloomFilter.contains(key, 0, key.length, bloom);
1338 }
1339
1340 return exists;
1341 }
1342 } catch (IOException e) {
1343 LOG.error("Error reading bloom filter data -- proceeding without",
1344 e);
1345 setGeneralBloomFilterFaulty();
1346 } catch (IllegalArgumentException e) {
1347 LOG.error("Bad bloom filter data -- proceeding without", e);
1348 setGeneralBloomFilterFaulty();
1349 }
1350
1351 return true;
1352 }
1353
1354 public Map<byte[], byte[]> loadFileInfo() throws IOException {
1355 Map<byte [], byte []> fi = reader.loadFileInfo();
1356
1357 byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
1358 if (b != null) {
1359 bloomFilterType = BloomType.valueOf(Bytes.toString(b));
1360 }
1361
1362 lastBloomKey = fi.get(LAST_BLOOM_KEY);
1363 byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
1364 if (cnt != null) {
1365 deleteFamilyCnt = Bytes.toLong(cnt);
1366 }
1367
1368 return fi;
1369 }
1370
1371 public void loadBloomfilter() {
1372 this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
1373 this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
1374 }
1375
1376 private void loadBloomfilter(BlockType blockType) {
1377 try {
1378 if (blockType == BlockType.GENERAL_BLOOM_META) {
1379 if (this.generalBloomFilter != null)
1380 return;
1381
1382 DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
1383 if (bloomMeta != null) {
1384
1385 if (bloomFilterType == BloomType.NONE) {
1386 throw new IOException(
1387 "valid bloom filter type not found in FileInfo");
1388 } else {
1389 generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
1390 reader);
1391 LOG.info("Loaded " + bloomFilterType.toString() + " ("
1392 + generalBloomFilter.getClass().getSimpleName()
1393 + ") metadata for " + reader.getName());
1394 }
1395 }
1396 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1397 if (this.deleteFamilyBloomFilter != null)
1398 return;
1399
1400 DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
1401 if (bloomMeta != null) {
1402 deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
1403 bloomMeta, reader);
1404 LOG.info("Loaded Delete Family Bloom ("
1405 + deleteFamilyBloomFilter.getClass().getSimpleName()
1406 + ") metadata for " + reader.getName());
1407 }
1408 } else {
1409 throw new RuntimeException("Block Type: " + blockType.toString()
1410 + "is not supported for Bloom filter");
1411 }
1412 } catch (IOException e) {
1413 LOG.error("Error reading bloom filter meta for " + blockType
1414 + " -- proceeding without", e);
1415 setBloomFilterFaulty(blockType);
1416 } catch (IllegalArgumentException e) {
1417 LOG.error("Bad bloom filter meta " + blockType
1418 + " -- proceeding without", e);
1419 setBloomFilterFaulty(blockType);
1420 }
1421 }
1422
1423 private void setBloomFilterFaulty(BlockType blockType) {
1424 if (blockType == BlockType.GENERAL_BLOOM_META) {
1425 setGeneralBloomFilterFaulty();
1426 } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
1427 setDeleteFamilyBloomFilterFaulty();
1428 }
1429 }
1430
1431
1432
1433
1434
1435
1436
1437
1438 public long getFilterEntries() {
1439 return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
1440 : reader.getEntries();
1441 }
1442
1443 public void setGeneralBloomFilterFaulty() {
1444 generalBloomFilter = null;
1445 }
1446
1447 public void setDeleteFamilyBloomFilterFaulty() {
1448 this.deleteFamilyBloomFilter = null;
1449 }
1450
1451 public byte[] getLastKey() {
1452 return reader.getLastKey();
1453 }
1454
1455 public byte[] midkey() throws IOException {
1456 return reader.midkey();
1457 }
1458
1459 public long length() {
1460 return reader.length();
1461 }
1462
1463 public long getTotalUncompressedBytes() {
1464 return reader.getTrailer().getTotalUncompressedBytes();
1465 }
1466
1467 public long getEntries() {
1468 return reader.getEntries();
1469 }
1470
1471 public long getDeleteFamilyCnt() {
1472 return deleteFamilyCnt;
1473 }
1474
1475 public byte[] getFirstKey() {
1476 return reader.getFirstKey();
1477 }
1478
1479 public long indexSize() {
1480 return reader.indexSize();
1481 }
1482
1483 public BloomType getBloomFilterType() {
1484 return this.bloomFilterType;
1485 }
1486
1487 public long getSequenceID() {
1488 return sequenceID;
1489 }
1490
1491 public void setSequenceID(long sequenceID) {
1492 this.sequenceID = sequenceID;
1493 }
1494
1495 BloomFilter getGeneralBloomFilter() {
1496 return generalBloomFilter;
1497 }
1498
1499 long getUncompressedDataIndexSize() {
1500 return reader.getTrailer().getUncompressedDataIndexSize();
1501 }
1502
1503 public long getTotalBloomSize() {
1504 if (generalBloomFilter == null)
1505 return 0;
1506 return generalBloomFilter.getByteSize();
1507 }
1508
1509 public int getHFileVersion() {
1510 return reader.getTrailer().getMajorVersion();
1511 }
1512
1513 public HFile.Reader getHFileReader() {
1514 return reader;
1515 }
1516
1517 void disableBloomFilterForTesting() {
1518 generalBloomFilter = null;
1519 this.deleteFamilyBloomFilter = null;
1520 }
1521
1522 public long getMaxTimestamp() {
1523 return timeRangeTracker == null ? Long.MAX_VALUE : timeRangeTracker.maximumTimestamp;
1524 }
1525 }
1526
1527
1528
1529
1530 public abstract static class Comparators {
1531
1532
1533
1534
1535
1536
1537
1538
1539 public static final Comparator<StoreFile> SEQ_ID =
1540 Ordering.compound(ImmutableList.of(
1541 Ordering.natural().onResultOf(new GetSeqId()),
1542 Ordering.natural().onResultOf(new GetFileSize()).reverse(),
1543 Ordering.natural().onResultOf(new GetBulkTime()),
1544 Ordering.natural().onResultOf(new GetPathName())
1545 ));
1546
1547 private static class GetSeqId implements Function<StoreFile, Long> {
1548 @Override
1549 public Long apply(StoreFile sf) {
1550 return sf.getMaxSequenceId();
1551 }
1552 }
1553
1554 private static class GetFileSize implements Function<StoreFile, Long> {
1555 @Override
1556 public Long apply(StoreFile sf) {
1557 return sf.getReader().length();
1558 }
1559 }
1560
1561 private static class GetBulkTime implements Function<StoreFile, Long> {
1562 @Override
1563 public Long apply(StoreFile sf) {
1564 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1565 return sf.getBulkLoadTimestamp();
1566 }
1567 }
1568
1569 private static class GetPathName implements Function<StoreFile, String> {
1570 @Override
1571 public String apply(StoreFile sf) {
1572 return sf.getPath().getName();
1573 }
1574 }
1575 }
1576 }