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