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