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