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