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