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