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 com.google.common.base.Function;
22 import com.google.common.base.Preconditions;
23 import com.google.common.collect.ImmutableList;
24 import com.google.common.collect.Ordering;
25
26 import java.io.DataInput;
27 import java.io.IOException;
28 import java.net.InetSocketAddress;
29 import java.nio.ByteBuffer;
30 import java.util.Arrays;
31 import java.util.Collection;
32 import java.util.Collections;
33 import java.util.Comparator;
34 import java.util.Map;
35 import java.util.SortedSet;
36 import java.util.UUID;
37 import java.util.concurrent.atomic.AtomicBoolean;
38
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.HConstants;
45 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
46 import org.apache.hadoop.hbase.KeyValue;
47 import org.apache.hadoop.hbase.KeyValue.KVComparator;
48 import org.apache.hadoop.hbase.classification.InterfaceAudience;
49 import org.apache.hadoop.hbase.client.Scan;
50 import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
51 import org.apache.hadoop.hbase.io.TimeRange;
52 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
53 import org.apache.hadoop.hbase.io.hfile.BlockType;
54 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
55 import org.apache.hadoop.hbase.io.hfile.HFile;
56 import org.apache.hadoop.hbase.io.hfile.HFileContext;
57 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
58 import org.apache.hadoop.hbase.io.hfile.HFileWriterV2;
59 import org.apache.hadoop.hbase.regionserver.compactions.Compactor;
60 import org.apache.hadoop.hbase.util.BloomFilter;
61 import org.apache.hadoop.hbase.util.BloomFilterFactory;
62 import org.apache.hadoop.hbase.util.BloomFilterWriter;
63 import org.apache.hadoop.hbase.util.Bytes;
64 import org.apache.hadoop.hbase.util.ChecksumType;
65 import org.apache.hadoop.hbase.util.Writables;
66 import org.apache.hadoop.io.WritableUtils;
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81 @InterfaceAudience.LimitedPrivate("Coprocessor")
82 public class StoreFile {
83 static final Log LOG = LogFactory.getLog(StoreFile.class.getName());
84
85
86
87
88 public static final byte [] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
89
90
91 public static final byte[] MAJOR_COMPACTION_KEY =
92 Bytes.toBytes("MAJOR_COMPACTION_KEY");
93
94
95 public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
96 Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
97
98
99 public static final byte[] BLOOM_FILTER_TYPE_KEY =
100 Bytes.toBytes("BLOOM_FILTER_TYPE");
101
102
103 public static final byte[] DELETE_FAMILY_COUNT =
104 Bytes.toBytes("DELETE_FAMILY_COUNT");
105
106
107 private static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
108
109
110 public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
111
112
113 public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
114
115 private final StoreFileInfo fileInfo;
116 private final FileSystem fs;
117
118
119 private final CacheConfig cacheConf;
120
121
122
123 private long sequenceid = -1;
124
125
126
127 private long maxMemstoreTS = -1;
128
129 public long getMaxMemstoreTS() {
130 return maxMemstoreTS;
131 }
132
133 public void setMaxMemstoreTS(long maxMemstoreTS) {
134 this.maxMemstoreTS = maxMemstoreTS;
135 }
136
137
138
139 private AtomicBoolean majorCompaction = null;
140
141
142
143 private boolean excludeFromMinorCompaction = false;
144
145
146 public static final byte[] BULKLOAD_TASK_KEY =
147 Bytes.toBytes("BULKLOAD_SOURCE_TASK");
148 public static final byte[] BULKLOAD_TIME_KEY =
149 Bytes.toBytes("BULKLOAD_TIMESTAMP");
150
151
152
153
154 private Map<byte[], byte[]> metadataMap;
155
156
157 private volatile Reader reader;
158
159
160
161
162
163 private final BloomType cfBloomType;
164
165
166 private long modificationTimeStamp = 0L;
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183 public StoreFile(final FileSystem fs, final Path p, final Configuration conf,
184 final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
185 this(fs, new StoreFileInfo(conf, fs, p), conf, cacheConf, cfBloomType);
186 }
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204 public StoreFile(final FileSystem fs, final StoreFileInfo fileInfo, final Configuration conf,
205 final CacheConfig cacheConf, final BloomType cfBloomType) throws IOException {
206 this.fs = fs;
207 this.fileInfo = fileInfo;
208 this.cacheConf = cacheConf;
209
210 if (BloomFilterFactory.isGeneralBloomEnabled(conf)) {
211 this.cfBloomType = cfBloomType;
212 } else {
213 LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " +
214 "cfBloomType=" + cfBloomType + " (disabled in config)");
215 this.cfBloomType = BloomType.NONE;
216 }
217
218
219 this.modificationTimeStamp = fileInfo.getModificationTime();
220 }
221
222
223
224
225
226 public StoreFile(final StoreFile other) {
227 this.fs = other.fs;
228 this.fileInfo = other.fileInfo;
229 this.cacheConf = other.cacheConf;
230 this.cfBloomType = other.cfBloomType;
231 this.modificationTimeStamp = other.modificationTimeStamp;
232 }
233
234
235
236
237 public StoreFile cloneForReader() {
238 return new StoreFile(this);
239 }
240
241
242
243
244
245 public StoreFileInfo getFileInfo() {
246 return this.fileInfo;
247 }
248
249
250
251
252 public Path getPath() {
253 return this.fileInfo.getPath();
254 }
255
256
257
258
259 public Path getQualifiedPath() {
260 return this.fileInfo.getPath().makeQualified(fs);
261 }
262
263
264
265
266
267 public boolean isReference() {
268 return this.fileInfo.isReference();
269 }
270
271
272
273
274 public boolean isHFile() {
275 return this.fileInfo.isHFile(this.fileInfo.getPath());
276 }
277
278
279
280
281 public boolean isMajorCompaction() {
282 if (this.majorCompaction == null) {
283 throw new NullPointerException("This has not been set yet");
284 }
285 return this.majorCompaction.get();
286 }
287
288
289
290
291 public boolean excludeFromMinorCompaction() {
292 return this.excludeFromMinorCompaction;
293 }
294
295
296
297
298 public long getMaxSequenceId() {
299 return this.sequenceid;
300 }
301
302 public long getModificationTimeStamp() {
303 return modificationTimeStamp;
304 }
305
306 public byte[] getMetadataValue(byte[] key) {
307 return metadataMap.get(key);
308 }
309
310
311
312
313
314
315
316
317
318 public static long getMaxMemstoreTSInList(Collection<StoreFile> sfs) {
319 long max = 0;
320 for (StoreFile sf : sfs) {
321 if (!sf.isBulkLoadResult()) {
322 max = Math.max(max, sf.getMaxMemstoreTS());
323 }
324 }
325 return max;
326 }
327
328
329
330
331
332
333
334
335 public static long getMaxSequenceIdInList(Collection<StoreFile> sfs) {
336 long max = 0;
337 for (StoreFile sf : sfs) {
338 max = Math.max(max, sf.getMaxSequenceId());
339 }
340 return max;
341 }
342
343 public CacheConfig getCacheConf() {
344 return this.cacheConf;
345 }
346
347
348
349
350
351
352
353
354
355
356
357 public boolean isBulkLoadResult() {
358 boolean bulkLoadedHFile = false;
359 String fileName = this.getPath().getName();
360 int startPos = fileName.indexOf("SeqId_");
361 if (startPos != -1) {
362 bulkLoadedHFile = true;
363 }
364 return metadataMap.containsKey(BULKLOAD_TIME_KEY) || bulkLoadedHFile;
365 }
366
367
368
369
370 public long getBulkLoadTimestamp() {
371 byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
372 return (bulkLoadTimestamp == null) ? 0 : Bytes.toLong(bulkLoadTimestamp);
373 }
374
375
376
377
378
379 public HDFSBlocksDistribution getHDFSBlockDistribution() {
380 return this.fileInfo.getHDFSBlockDistribution();
381 }
382
383
384
385
386
387
388
389 private Reader open(boolean canUseDropBehind) throws IOException {
390 if (this.reader != null) {
391 throw new IllegalAccessError("Already open");
392 }
393
394
395 this.reader = fileInfo.open(this.fs, this.cacheConf, canUseDropBehind);
396
397
398 metadataMap = Collections.unmodifiableMap(this.reader.loadFileInfo());
399
400
401 byte [] b = metadataMap.get(MAX_SEQ_ID_KEY);
402 if (b != null) {
403
404
405
406
407
408 this.sequenceid = Bytes.toLong(b);
409 if (fileInfo.isTopReference()) {
410 this.sequenceid += 1;
411 }
412 }
413
414 if (isBulkLoadResult()){
415
416
417 String fileName = this.getPath().getName();
418
419 int startPos = fileName.lastIndexOf("SeqId_");
420 if (startPos != -1) {
421 this.sequenceid = Long.parseLong(fileName.substring(startPos + 6,
422 fileName.indexOf('_', startPos + 6)));
423
424 if (fileInfo.isTopReference()) {
425 this.sequenceid += 1;
426 }
427 }
428 this.reader.setBulkLoaded(true);
429 }
430 this.reader.setSequenceID(this.sequenceid);
431
432 b = metadataMap.get(HFileWriterV2.MAX_MEMSTORE_TS_KEY);
433 if (b != null) {
434 this.maxMemstoreTS = Bytes.toLong(b);
435 }
436
437 b = metadataMap.get(MAJOR_COMPACTION_KEY);
438 if (b != null) {
439 boolean mc = Bytes.toBoolean(b);
440 if (this.majorCompaction == null) {
441 this.majorCompaction = new AtomicBoolean(mc);
442 } else {
443 this.majorCompaction.set(mc);
444 }
445 } else {
446
447
448 this.majorCompaction = new AtomicBoolean(false);
449 }
450
451 b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY);
452 this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b));
453
454 BloomType hfileBloomType = reader.getBloomFilterType();
455 if (cfBloomType != BloomType.NONE) {
456 reader.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
457 if (hfileBloomType != cfBloomType) {
458 LOG.info("HFile Bloom filter type for "
459 + reader.getHFileReader().getName() + ": " + hfileBloomType
460 + ", but " + cfBloomType + " specified in column family "
461 + "configuration");
462 }
463 } else if (hfileBloomType != BloomType.NONE) {
464 LOG.info("Bloom filter turned off by CF config for "
465 + reader.getHFileReader().getName());
466 }
467
468
469 reader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
470
471 try {
472 this.reader.timeRange = TimeRangeTracker.getTimeRange(metadataMap.get(TIMERANGE_KEY));
473 } catch (IllegalArgumentException e) {
474 LOG.error("Error reading timestamp range data from meta -- " +
475 "proceeding without", e);
476 this.reader.timeRange = null;
477 }
478 return this.reader;
479 }
480
481 public Reader createReader() throws IOException {
482 return createReader(false);
483 }
484
485
486
487
488
489 public Reader createReader(boolean canUseDropBehind) throws IOException {
490 if (this.reader == null) {
491 try {
492 this.reader = open(canUseDropBehind);
493 } catch (IOException e) {
494 try {
495 boolean evictOnClose =
496 cacheConf != null? cacheConf.shouldEvictOnClose(): true;
497 this.closeReader(evictOnClose);
498 } catch (IOException ee) {
499 }
500 throw e;
501 }
502
503 }
504 return this.reader;
505 }
506
507
508
509
510
511 public Reader getReader() {
512 return this.reader;
513 }
514
515
516
517
518
519 public synchronized void closeReader(boolean evictOnClose)
520 throws IOException {
521 if (this.reader != null) {
522 this.reader.close(evictOnClose);
523 this.reader = null;
524 }
525 }
526
527
528
529
530
531 public void deleteReader() throws IOException {
532 boolean evictOnClose =
533 cacheConf != null? cacheConf.shouldEvictOnClose(): true;
534 closeReader(evictOnClose);
535 this.fs.delete(getPath(), true);
536 }
537
538 @Override
539 public String toString() {
540 return this.fileInfo.toString();
541 }
542
543
544
545
546 public String toStringDetailed() {
547 StringBuilder sb = new StringBuilder();
548 sb.append(this.getPath().toString());
549 sb.append(", isReference=").append(isReference());
550 sb.append(", isBulkLoadResult=").append(isBulkLoadResult());
551 if (isBulkLoadResult()) {
552 sb.append(", bulkLoadTS=").append(getBulkLoadTimestamp());
553 } else {
554 sb.append(", seqid=").append(getMaxSequenceId());
555 }
556 sb.append(", majorCompaction=").append(isMajorCompaction());
557
558 return sb.toString();
559 }
560
561 public static class WriterBuilder {
562 private final Configuration conf;
563 private final CacheConfig cacheConf;
564 private final FileSystem fs;
565
566 private KeyValue.KVComparator comparator = KeyValue.COMPARATOR;
567 private BloomType bloomType = BloomType.NONE;
568 private long maxKeyCount = 0;
569 private Path dir;
570 private Path filePath;
571 private InetSocketAddress[] favoredNodes;
572 private HFileContext fileContext;
573 private boolean shouldDropCacheBehind = false;
574
575 public WriterBuilder(Configuration conf, CacheConfig cacheConf,
576 FileSystem fs) {
577 this.conf = conf;
578 this.cacheConf = cacheConf;
579 this.fs = fs;
580 }
581
582
583
584
585
586
587
588
589 public WriterBuilder withOutputDir(Path dir) {
590 Preconditions.checkNotNull(dir);
591 this.dir = dir;
592 return this;
593 }
594
595
596
597
598
599
600 public WriterBuilder withFilePath(Path filePath) {
601 Preconditions.checkNotNull(filePath);
602 this.filePath = filePath;
603 return this;
604 }
605
606
607
608
609
610 public WriterBuilder withFavoredNodes(InetSocketAddress[] favoredNodes) {
611 this.favoredNodes = favoredNodes;
612 return this;
613 }
614
615 public WriterBuilder withComparator(KeyValue.KVComparator comparator) {
616 Preconditions.checkNotNull(comparator);
617 this.comparator = comparator;
618 return this;
619 }
620
621 public WriterBuilder withBloomType(BloomType bloomType) {
622 Preconditions.checkNotNull(bloomType);
623 this.bloomType = bloomType;
624 return this;
625 }
626
627
628
629
630
631 public WriterBuilder withMaxKeyCount(long maxKeyCount) {
632 this.maxKeyCount = maxKeyCount;
633 return this;
634 }
635
636 public WriterBuilder withFileContext(HFileContext fileContext) {
637 this.fileContext = fileContext;
638 return this;
639 }
640
641 public WriterBuilder withShouldDropCacheBehind(boolean shouldDropCacheBehind) {
642 this.shouldDropCacheBehind = shouldDropCacheBehind;
643 return this;
644 }
645
646
647
648
649
650 public Writer build() throws IOException {
651 if ((dir == null ? 0 : 1) + (filePath == null ? 0 : 1) != 1) {
652 throw new IllegalArgumentException("Either specify parent directory " +
653 "or file path");
654 }
655
656 if (dir == null) {
657 dir = filePath.getParent();
658 }
659
660 if (!fs.exists(dir)) {
661 fs.mkdirs(dir);
662 }
663
664 if (filePath == null) {
665 filePath = getUniqueFile(fs, dir);
666 if (!BloomFilterFactory.isGeneralBloomEnabled(conf)) {
667 bloomType = BloomType.NONE;
668 }
669 }
670
671 if (comparator == null) {
672 comparator = KeyValue.COMPARATOR;
673 }
674 return new Writer(fs, filePath,
675 conf, cacheConf, comparator, bloomType, maxKeyCount, favoredNodes, fileContext);
676 }
677 }
678
679
680
681
682
683
684 public static Path getUniqueFile(final FileSystem fs, final Path dir)
685 throws IOException {
686 if (!fs.getFileStatus(dir).isDir()) {
687 throw new IOException("Expecting " + dir.toString() +
688 " to be a directory");
689 }
690 return new Path(dir, UUID.randomUUID().toString().replaceAll("-", ""));
691 }
692
693 public Long getMinimumTimestamp() {
694 return getReader().timeRange == null? null: getReader().timeRange.getMin();
695 }
696
697 public Long getMaximumTimestamp() {
698 return getReader().timeRange == null? null: getReader().timeRange.getMax();
699 }
700
701
702
703
704
705
706 @SuppressWarnings("deprecation")
707 byte[] getFileSplitPoint(KVComparator comparator) throws IOException {
708 if (this.reader == null) {
709 LOG.warn("Storefile " + this + " Reader is null; cannot get split point");
710 return null;
711 }
712
713
714
715 byte [] midkey = this.reader.midkey();
716 if (midkey != null) {
717 KeyValue mk = KeyValue.createKeyValueFromKey(midkey, 0, midkey.length);
718 byte [] fk = this.reader.getFirstKey();
719 KeyValue firstKey = KeyValue.createKeyValueFromKey(fk, 0, fk.length);
720 byte [] lk = this.reader.getLastKey();
721 KeyValue lastKey = KeyValue.createKeyValueFromKey(lk, 0, lk.length);
722
723 if (comparator.compareRows(mk, firstKey) == 0 || comparator.compareRows(mk, lastKey) == 0) {
724 if (LOG.isDebugEnabled()) {
725 LOG.debug("cannot split because midkey is the same as first or last row");
726 }
727 return null;
728 }
729 return mk.getRow();
730 }
731 return null;
732 }
733
734
735
736
737
738 public static class Writer implements Compactor.CellSink {
739 private final BloomFilterWriter generalBloomFilterWriter;
740 private final BloomFilterWriter deleteFamilyBloomFilterWriter;
741 private final BloomType bloomType;
742 private byte[] lastBloomKey;
743 private int lastBloomKeyOffset, lastBloomKeyLen;
744 private KVComparator kvComparator;
745 private KeyValue lastKv = null;
746 private long earliestPutTs = HConstants.LATEST_TIMESTAMP;
747 private KeyValue lastDeleteFamilyKV = null;
748 private long deleteFamilyCnt = 0;
749
750
751
752 protected ChecksumType checksumType;
753
754
755 protected int bytesPerChecksum;
756
757 TimeRangeTracker timeRangeTracker = new TimeRangeTracker();
758
759
760
761
762
763
764
765 boolean timeRangeTrackerSet = 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
853
854
855
856 public void setTimeRangeTracker(final TimeRangeTracker trt) {
857 this.timeRangeTracker = trt;
858 timeRangeTrackerSet = true;
859 }
860
861
862
863
864
865
866
867
868 public void trackTimestamps(final KeyValue kv) {
869 if (KeyValue.Type.Put.getCode() == kv.getTypeByte()) {
870 earliestPutTs = Math.min(earliestPutTs, kv.getTimestamp());
871 }
872 if (!timeRangeTrackerSet) {
873 timeRangeTracker.includeTimestamp(kv);
874 }
875 }
876
877 private void appendGeneralBloomfilter(final KeyValue kv) throws IOException {
878 if (this.generalBloomFilterWriter != null) {
879
880 boolean newKey = true;
881 if (this.lastKv != null) {
882 switch(bloomType) {
883 case ROW:
884 newKey = ! kvComparator.matchingRows(kv, lastKv);
885 break;
886 case ROWCOL:
887 newKey = ! kvComparator.matchingRowColumn(kv, lastKv);
888 break;
889 case NONE:
890 newKey = false;
891 break;
892 default:
893 throw new IOException("Invalid Bloom filter type: " + bloomType +
894 " (ROW or ROWCOL expected)");
895 }
896 }
897 if (newKey) {
898
899
900
901
902
903
904
905
906 byte[] bloomKey;
907 int bloomKeyOffset, bloomKeyLen;
908
909 switch (bloomType) {
910 case ROW:
911 bloomKey = kv.getBuffer();
912 bloomKeyOffset = kv.getRowOffset();
913 bloomKeyLen = kv.getRowLength();
914 break;
915 case ROWCOL:
916
917
918
919 bloomKey = generalBloomFilterWriter.createBloomKey(kv.getBuffer(),
920 kv.getRowOffset(), kv.getRowLength(), kv.getBuffer(),
921 kv.getQualifierOffset(), kv.getQualifierLength());
922 bloomKeyOffset = 0;
923 bloomKeyLen = bloomKey.length;
924 break;
925 default:
926 throw new IOException("Invalid Bloom filter type: " + bloomType +
927 " (ROW or ROWCOL expected)");
928 }
929 generalBloomFilterWriter.add(bloomKey, bloomKeyOffset, bloomKeyLen);
930 if (lastBloomKey != null
931 && generalBloomFilterWriter.getComparator().compareFlatKey(bloomKey,
932 bloomKeyOffset, bloomKeyLen, lastBloomKey,
933 lastBloomKeyOffset, lastBloomKeyLen) <= 0) {
934 throw new IOException("Non-increasing Bloom keys: "
935 + Bytes.toStringBinary(bloomKey, bloomKeyOffset, bloomKeyLen)
936 + " after "
937 + Bytes.toStringBinary(lastBloomKey, lastBloomKeyOffset,
938 lastBloomKeyLen));
939 }
940 lastBloomKey = bloomKey;
941 lastBloomKeyOffset = bloomKeyOffset;
942 lastBloomKeyLen = bloomKeyLen;
943 this.lastKv = kv;
944 }
945 }
946 }
947
948 private void appendDeleteFamilyBloomFilter(final KeyValue kv)
949 throws IOException {
950 if (!kv.isDeleteFamily() && !kv.isDeleteFamilyVersion()) {
951 return;
952 }
953
954
955 deleteFamilyCnt++;
956 if (null != this.deleteFamilyBloomFilterWriter) {
957 boolean newKey = true;
958 if (lastDeleteFamilyKV != null) {
959 newKey = !kvComparator.matchingRows(kv, lastDeleteFamilyKV);
960 }
961 if (newKey) {
962 this.deleteFamilyBloomFilterWriter.add(kv.getBuffer(),
963 kv.getRowOffset(), kv.getRowLength());
964 this.lastDeleteFamilyKV = kv;
965 }
966 }
967 }
968
969 public void append(final KeyValue kv) throws IOException {
970 appendGeneralBloomfilter(kv);
971 appendDeleteFamilyBloomFilter(kv);
972 writer.append(kv);
973 trackTimestamps(kv);
974 }
975
976 public Path getPath() {
977 return this.writer.getPath();
978 }
979
980 boolean hasGeneralBloom() {
981 return this.generalBloomFilterWriter != null;
982 }
983
984
985
986
987
988
989 BloomFilterWriter getGeneralBloomWriter() {
990 return generalBloomFilterWriter;
991 }
992
993 private boolean closeBloomFilter(BloomFilterWriter bfw) throws IOException {
994 boolean haveBloom = (bfw != null && bfw.getKeyCount() > 0);
995 if (haveBloom) {
996 bfw.compactBloom();
997 }
998 return haveBloom;
999 }
1000
1001 private boolean closeGeneralBloomFilter() throws IOException {
1002 boolean hasGeneralBloom = closeBloomFilter(generalBloomFilterWriter);
1003
1004
1005 if (hasGeneralBloom) {
1006 writer.addGeneralBloomFilter(generalBloomFilterWriter);
1007 writer.appendFileInfo(BLOOM_FILTER_TYPE_KEY,
1008 Bytes.toBytes(bloomType.toString()));
1009 if (lastBloomKey != null) {
1010 writer.appendFileInfo(LAST_BLOOM_KEY, Arrays.copyOfRange(
1011 lastBloomKey, lastBloomKeyOffset, lastBloomKeyOffset
1012 + lastBloomKeyLen));
1013 }
1014 }
1015 return hasGeneralBloom;
1016 }
1017
1018 private boolean closeDeleteFamilyBloomFilter() throws IOException {
1019 boolean hasDeleteFamilyBloom = closeBloomFilter(deleteFamilyBloomFilterWriter);
1020
1021
1022 if (hasDeleteFamilyBloom) {
1023 writer.addDeleteFamilyBloomFilter(deleteFamilyBloomFilterWriter);
1024 }
1025
1026
1027
1028 writer.appendFileInfo(DELETE_FAMILY_COUNT,
1029 Bytes.toBytes(this.deleteFamilyCnt));
1030
1031 return hasDeleteFamilyBloom;
1032 }
1033
1034 public void close() throws IOException {
1035 boolean hasGeneralBloom = this.closeGeneralBloomFilter();
1036 boolean hasDeleteFamilyBloom = this.closeDeleteFamilyBloomFilter();
1037
1038 writer.close();
1039
1040
1041
1042 if (StoreFile.LOG.isTraceEnabled()) {
1043 StoreFile.LOG.trace((hasGeneralBloom ? "" : "NO ") + "General Bloom and " +
1044 (hasDeleteFamilyBloom ? "" : "NO ") + "DeleteFamily" + " was added to HFile " +
1045 getPath());
1046 }
1047
1048 }
1049
1050 public void appendFileInfo(byte[] key, byte[] value) throws IOException {
1051 writer.appendFileInfo(key, value);
1052 }
1053
1054
1055
1056 HFile.Writer getHFileWriter() {
1057 return writer;
1058 }
1059 }
1060
1061
1062
1063
1064 public static class Reader {
1065 static final Log LOG = LogFactory.getLog(Reader.class.getName());
1066
1067 protected BloomFilter generalBloomFilter = null;
1068 protected BloomFilter deleteFamilyBloomFilter = null;
1069 protected BloomType bloomFilterType;
1070 private final HFile.Reader reader;
1071 protected TimeRange timeRange;
1072 protected long sequenceID = -1;
1073 private byte[] lastBloomKey;
1074 private long deleteFamilyCnt = -1;
1075 private boolean bulkLoadResult = false;
1076
1077 public Reader(FileSystem fs, Path path, CacheConfig cacheConf, Configuration conf)
1078 throws IOException {
1079 reader = HFile.createReader(fs, path, cacheConf, conf);
1080 bloomFilterType = BloomType.NONE;
1081 }
1082
1083 public Reader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
1084 CacheConfig cacheConf, Configuration conf) throws IOException {
1085 reader = HFile.createReader(fs, path, in, size, cacheConf, conf);
1086 bloomFilterType = BloomType.NONE;
1087 }
1088
1089
1090
1091
1092 Reader() {
1093 this.reader = null;
1094 }
1095
1096 public KVComparator getComparator() {
1097 return reader.getComparator();
1098 }
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1109 boolean pread) {
1110 return getStoreFileScanner(cacheBlocks, pread, false,
1111
1112
1113 0);
1114 }
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127 public StoreFileScanner getStoreFileScanner(boolean cacheBlocks,
1128 boolean pread,
1129 boolean isCompaction, long readPt) {
1130 return new StoreFileScanner(this,
1131 getScanner(cacheBlocks, pread, isCompaction),
1132 !isCompaction, reader.hasMVCCInfo() && !this.bulkLoadResult,
1133 readPt);
1134 }
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145 @Deprecated
1146 public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
1147 return getScanner(cacheBlocks, pread, false);
1148 }
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163 @Deprecated
1164 public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
1165 boolean isCompaction) {
1166 return reader.getScanner(cacheBlocks, pread, isCompaction);
1167 }
1168
1169 public void close(boolean evictOnClose) throws IOException {
1170 reader.close(evictOnClose);
1171 }
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181 boolean passesTimerangeFilter(TimeRange tr, long oldestUnexpiredTS) {
1182 return this.timeRange == null? true:
1183 this.timeRange.includesTimeRange(tr) && this.timeRange.getMax() >= oldestUnexpiredTS;
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 timeRange == null ? Long.MAX_VALUE : timeRange.getMax();
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
1608
1609
1610
1611 public static final Comparator<StoreFile> SEQ_ID_MAX_TIMESTAMP =
1612 Ordering.compound(ImmutableList.of(
1613 Ordering.natural().onResultOf(new GetSeqId()),
1614 Ordering.natural().onResultOf(new GetMaxTimestamp()),
1615 Ordering.natural().onResultOf(new GetFileSize()).reverse(),
1616 Ordering.natural().onResultOf(new GetBulkTime()),
1617 Ordering.natural().onResultOf(new GetPathName())
1618 ));
1619
1620 private static class GetSeqId implements Function<StoreFile, Long> {
1621 @Override
1622 public Long apply(StoreFile sf) {
1623 return sf.getMaxSequenceId();
1624 }
1625 }
1626
1627 private static class GetFileSize implements Function<StoreFile, Long> {
1628 @Override
1629 public Long apply(StoreFile sf) {
1630 return sf.getReader().length();
1631 }
1632 }
1633
1634 private static class GetBulkTime implements Function<StoreFile, Long> {
1635 @Override
1636 public Long apply(StoreFile sf) {
1637 if (!sf.isBulkLoadResult()) return Long.MAX_VALUE;
1638 return sf.getBulkLoadTimestamp();
1639 }
1640 }
1641
1642 private static class GetPathName implements Function<StoreFile, String> {
1643 @Override
1644 public String apply(StoreFile sf) {
1645 return sf.getPath().getName();
1646 }
1647 }
1648
1649 private static class GetMaxTimestamp implements Function<StoreFile, Long> {
1650 @Override
1651 public Long apply(StoreFile sf) {
1652 return sf.getMaximumTimestamp() == null? (Long)Long.MAX_VALUE : sf.getMaximumTimestamp();
1653 }
1654 }
1655 }
1656 }