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