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