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