001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.regionserver;
019
020import static org.apache.hadoop.hbase.regionserver.HStoreFile.BLOOM_FILTER_TYPE_KEY;
021import static org.apache.hadoop.hbase.regionserver.HStoreFile.DELETE_FAMILY_COUNT;
022import static org.apache.hadoop.hbase.regionserver.HStoreFile.LAST_BLOOM_KEY;
023
024import java.io.DataInput;
025import java.io.IOException;
026import java.util.Map;
027import java.util.Optional;
028import java.util.SortedSet;
029import java.util.concurrent.atomic.AtomicInteger;
030
031import org.apache.hadoop.conf.Configuration;
032import org.apache.hadoop.fs.FileSystem;
033import org.apache.hadoop.fs.Path;
034import org.apache.hadoop.hbase.Cell;
035import org.apache.hadoop.hbase.CellComparator;
036import org.apache.hadoop.hbase.HBaseInterfaceAudience;
037import org.apache.hadoop.hbase.HConstants;
038import org.apache.hadoop.hbase.PrivateCellUtil;
039import org.apache.hadoop.hbase.KeyValue;
040import org.apache.hadoop.hbase.client.Scan;
041import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
042import org.apache.hadoop.hbase.io.TimeRange;
043import org.apache.hadoop.hbase.io.hfile.BlockType;
044import org.apache.hadoop.hbase.io.hfile.CacheConfig;
045import org.apache.hadoop.hbase.io.hfile.HFile;
046import org.apache.hadoop.hbase.io.hfile.HFileBlock;
047import org.apache.hadoop.hbase.io.hfile.HFileScanner;
048import org.apache.hadoop.hbase.nio.ByteBuff;
049import org.apache.hadoop.hbase.util.BloomFilter;
050import org.apache.hadoop.hbase.util.BloomFilterFactory;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.apache.yetus.audience.InterfaceAudience;
053import org.apache.yetus.audience.InterfaceStability;
054import org.slf4j.Logger;
055import org.slf4j.LoggerFactory;
056import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
057
058/**
059 * Reader for a StoreFile.
060 */
061@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.PHOENIX)
062@InterfaceStability.Evolving
063public class StoreFileReader {
064  private static final Logger LOG = LoggerFactory.getLogger(StoreFileReader.class.getName());
065
066  protected BloomFilter generalBloomFilter = null;
067  protected BloomFilter deleteFamilyBloomFilter = null;
068  protected BloomType bloomFilterType;
069  private final HFile.Reader reader;
070  protected long sequenceID = -1;
071  protected TimeRange timeRange = null;
072  private byte[] lastBloomKey;
073  private long deleteFamilyCnt = -1;
074  private boolean bulkLoadResult = false;
075  private KeyValue.KeyOnlyKeyValue lastBloomKeyOnlyKV = null;
076  private boolean skipResetSeqId = true;
077
078  // Counter that is incremented every time a scanner is created on the
079  // store file. It is decremented when the scan on the store file is
080  // done. All StoreFileReader for the same StoreFile will share this counter.
081  private final AtomicInteger refCount;
082
083  // indicate that whether this StoreFileReader is shared, i.e., used for pread. If not, we will
084  // close the internal reader when readCompleted is called.
085  @VisibleForTesting
086  final boolean shared;
087
088  private volatile Listener listener;
089
090  private boolean closed = false;
091
092  private StoreFileReader(HFile.Reader reader, AtomicInteger refCount, boolean shared) {
093    this.reader = reader;
094    bloomFilterType = BloomType.NONE;
095    this.refCount = refCount;
096    this.shared = shared;
097  }
098
099  public StoreFileReader(FileSystem fs, Path path, CacheConfig cacheConf,
100      boolean primaryReplicaStoreFile, AtomicInteger refCount, boolean shared, Configuration conf)
101      throws IOException {
102    this(HFile.createReader(fs, path, cacheConf, primaryReplicaStoreFile, conf), refCount, shared);
103  }
104
105  public StoreFileReader(FileSystem fs, Path path, FSDataInputStreamWrapper in, long size,
106      CacheConfig cacheConf, boolean primaryReplicaStoreFile, AtomicInteger refCount,
107      boolean shared, Configuration conf) throws IOException {
108    this(HFile.createReader(fs, path, in, size, cacheConf, primaryReplicaStoreFile, conf), refCount,
109        shared);
110  }
111
112  void copyFields(StoreFileReader reader) {
113    this.generalBloomFilter = reader.generalBloomFilter;
114    this.deleteFamilyBloomFilter = reader.deleteFamilyBloomFilter;
115    this.bloomFilterType = reader.bloomFilterType;
116    this.sequenceID = reader.sequenceID;
117    this.timeRange = reader.timeRange;
118    this.lastBloomKey = reader.lastBloomKey;
119    this.bulkLoadResult = reader.bulkLoadResult;
120    this.lastBloomKeyOnlyKV = reader.lastBloomKeyOnlyKV;
121    this.skipResetSeqId = reader.skipResetSeqId;
122  }
123
124  public boolean isPrimaryReplicaReader() {
125    return reader.isPrimaryReplicaReader();
126  }
127
128  /**
129   * ONLY USE DEFAULT CONSTRUCTOR FOR UNIT TESTS
130   */
131  @VisibleForTesting
132  StoreFileReader() {
133    this.refCount = new AtomicInteger(0);
134    this.reader = null;
135    this.shared = false;
136  }
137
138  public CellComparator getComparator() {
139    return reader.getComparator();
140  }
141
142  /**
143   * Get a scanner to scan over this StoreFile.
144   * @param cacheBlocks should this scanner cache blocks?
145   * @param pread use pread (for highly concurrent small readers)
146   * @param isCompaction is scanner being used for compaction?
147   * @param scannerOrder Order of this scanner relative to other scanners. See
148   *          {@link KeyValueScanner#getScannerOrder()}.
149   * @param canOptimizeForNonNullColumn {@code true} if we can make sure there is no null column,
150   *          otherwise {@code false}. This is a hint for optimization.
151   * @return a scanner
152   */
153  public StoreFileScanner getStoreFileScanner(boolean cacheBlocks, boolean pread,
154      boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) {
155    return new StoreFileScanner(this, getScanner(cacheBlocks, pread, isCompaction),
156        !isCompaction, reader.hasMVCCInfo(), readPt, scannerOrder, canOptimizeForNonNullColumn);
157  }
158
159  /**
160   * Return the ref count associated with the reader whenever a scanner associated with the
161   * reader is opened.
162   */
163  int getRefCount() {
164    return refCount.get();
165  }
166
167  /**
168   * Indicate that the scanner has started reading with this reader. We need to increment the ref
169   * count so reader is not close until some object is holding the lock
170   */
171  void incrementRefCount() {
172    refCount.incrementAndGet();
173  }
174
175  /**
176   * Indicate that the scanner has finished reading with this reader. We need to decrement the ref
177   * count, and also, if this is not the common pread reader, we should close it.
178   */
179  void readCompleted() {
180    refCount.decrementAndGet();
181    if (!shared) {
182      try {
183        reader.close(false);
184        if (this.listener != null) {
185          this.listener.storeFileReaderClosed(this);
186        }
187      } catch (IOException e) {
188        LOG.warn("failed to close stream reader", e);
189      }
190    }
191  }
192
193  /**
194   * @deprecated since 2.0.0 and will be removed in 3.0.0. Do not write further code which depends
195   *   on this call. Instead use getStoreFileScanner() which uses the StoreFileScanner
196   *   class/interface which is the preferred way to scan a store with higher level concepts.
197   *
198   * @param cacheBlocks should we cache the blocks?
199   * @param pread use pread (for concurrent small readers)
200   * @return the underlying HFileScanner
201   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15296">HBASE-15296</a>
202   */
203  @Deprecated
204  public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
205    return getScanner(cacheBlocks, pread, false);
206  }
207
208  /**
209   * @deprecated since 2.0.0 and will be removed in 3.0.0. Do not write further code which depends
210   *   on this call. Instead use getStoreFileScanner() which uses the StoreFileScanner
211   *   class/interface which is the preferred way to scan a store with higher level concepts.
212   *
213   * @param cacheBlocks
214   *          should we cache the blocks?
215   * @param pread
216   *          use pread (for concurrent small readers)
217   * @param isCompaction
218   *          is scanner being used for compaction?
219   * @return the underlying HFileScanner
220   * @see <a href="https://issues.apache.org/jira/browse/HBASE-15296">HBASE-15296</a>
221   */
222  @Deprecated
223  public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
224      boolean isCompaction) {
225    return reader.getScanner(cacheBlocks, pread, isCompaction);
226  }
227
228  public void close(boolean evictOnClose) throws IOException {
229    synchronized (this) {
230      if (closed) {
231        return;
232      }
233      reader.close(evictOnClose);
234      closed = true;
235    }
236    if (listener != null) {
237      listener.storeFileReaderClosed(this);
238    }
239  }
240
241  /**
242   * Check if this storeFile may contain keys within the TimeRange that
243   * have not expired (i.e. not older than oldestUnexpiredTS).
244   * @param timeRange the timeRange to restrict
245   * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
246   *          determined by the column family's TTL
247   * @return false if queried keys definitely don't exist in this StoreFile
248   */
249  boolean passesTimerangeFilter(TimeRange tr, long oldestUnexpiredTS) {
250    return this.timeRange == null? true:
251      this.timeRange.includesTimeRange(tr) && this.timeRange.getMax() >= oldestUnexpiredTS;
252  }
253
254  /**
255   * Checks whether the given scan passes the Bloom filter (if present). Only
256   * checks Bloom filters for single-row or single-row-column scans. Bloom
257   * filter checking for multi-gets is implemented as part of the store
258   * scanner system (see {@link StoreFileScanner#seek(Cell)} and uses
259   * the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
260   * and {@link #passesGeneralRowColBloomFilter(Cell)}.
261   *
262   * @param scan the scan specification. Used to determine the row, and to
263   *          check whether this is a single-row ("get") scan.
264   * @param columns the set of columns. Only used for row-column Bloom
265   *          filters.
266   * @return true if the scan with the given column set passes the Bloom
267   *         filter, or if the Bloom filter is not applicable for the scan.
268   *         False if the Bloom filter is applicable and the scan fails it.
269   */
270  boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
271    // Multi-column non-get scans will use Bloom filters through the
272    // lower-level API function that this function calls.
273    if (!scan.isGetScan()) {
274      return true;
275    }
276
277    byte[] row = scan.getStartRow();
278    switch (this.bloomFilterType) {
279      case ROW:
280        return passesGeneralRowBloomFilter(row, 0, row.length);
281
282      case ROWCOL:
283        if (columns != null && columns.size() == 1) {
284          byte[] column = columns.first();
285          // create the required fake key
286          Cell kvKey = PrivateCellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
287          return passesGeneralRowColBloomFilter(kvKey);
288        }
289
290        // For multi-column queries the Bloom filter is checked from the
291        // seekExact operation.
292        return true;
293
294      default:
295        return true;
296    }
297  }
298
299  public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
300      int rowLen) {
301    // Cache Bloom filter as a local variable in case it is set to null by
302    // another thread on an IO error.
303    BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
304
305    // Empty file or there is no delete family at all
306    if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
307      return false;
308    }
309
310    if (bloomFilter == null) {
311      return true;
312    }
313
314    try {
315      if (!bloomFilter.supportsAutoLoading()) {
316        return true;
317      }
318      return bloomFilter.contains(row, rowOffset, rowLen, null);
319    } catch (IllegalArgumentException e) {
320      LOG.error("Bad Delete Family bloom filter data -- proceeding without",
321          e);
322      setDeleteFamilyBloomFilterFaulty();
323    }
324
325    return true;
326  }
327
328  /**
329   * A method for checking Bloom filters. Called directly from
330   * StoreFileScanner in case of a multi-column query.
331   *
332   * @return True if passes
333   */
334  public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
335    BloomFilter bloomFilter = this.generalBloomFilter;
336    if (bloomFilter == null) {
337      return true;
338    }
339
340    // Used in ROW bloom
341    byte[] key = null;
342    if (rowOffset != 0 || rowLen != row.length) {
343      throw new AssertionError(
344          "For row-only Bloom filters the row " + "must occupy the whole array");
345    }
346    key = row;
347    return checkGeneralBloomFilter(key, null, bloomFilter);
348  }
349
350  /**
351   * A method for checking Bloom filters. Called directly from
352   * StoreFileScanner in case of a multi-column query.
353   *
354   * @param cell
355   *          the cell to check if present in BloomFilter
356   * @return True if passes
357   */
358  public boolean passesGeneralRowColBloomFilter(Cell cell) {
359    BloomFilter bloomFilter = this.generalBloomFilter;
360    if (bloomFilter == null) {
361      return true;
362    }
363    // Used in ROW_COL bloom
364    Cell kvKey = null;
365    // Already if the incoming key is a fake rowcol key then use it as it is
366    if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
367      kvKey = cell;
368    } else {
369      kvKey = PrivateCellUtil.createFirstOnRowCol(cell);
370    }
371    return checkGeneralBloomFilter(null, kvKey, bloomFilter);
372  }
373
374  private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
375    // Empty file
376    if (reader.getTrailer().getEntryCount() == 0) {
377      return false;
378    }
379    HFileBlock bloomBlock = null;
380    try {
381      boolean shouldCheckBloom;
382      ByteBuff bloom;
383      if (bloomFilter.supportsAutoLoading()) {
384        bloom = null;
385        shouldCheckBloom = true;
386      } else {
387        bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
388        bloom = bloomBlock.getBufferWithoutHeader();
389        shouldCheckBloom = bloom != null;
390      }
391
392      if (shouldCheckBloom) {
393        boolean exists;
394
395        // Whether the primary Bloom key is greater than the last Bloom key
396        // from the file info. For row-column Bloom filters this is not yet
397        // a sufficient condition to return false.
398        boolean keyIsAfterLast = (lastBloomKey != null);
399        // hbase:meta does not have blooms. So we need not have special interpretation
400        // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
401        if (keyIsAfterLast) {
402          if (bloomFilterType == BloomType.ROW) {
403            keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
404          } else {
405            keyIsAfterLast = (CellComparator.getInstance().compare(kvKey, lastBloomKeyOnlyKV)) > 0;
406          }
407        }
408
409        if (bloomFilterType == BloomType.ROWCOL) {
410          // Since a Row Delete is essentially a DeleteFamily applied to all
411          // columns, a file might be skipped if using row+col Bloom filter.
412          // In order to ensure this file is included an additional check is
413          // required looking only for a row bloom.
414          Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);
415          // hbase:meta does not have blooms. So we need not have special interpretation
416          // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
417          if (keyIsAfterLast
418              && (CellComparator.getInstance().compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
419            exists = false;
420          } else {
421            exists =
422                bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL) ||
423                bloomFilter.contains(rowBloomKey, bloom, BloomType.ROWCOL);
424          }
425        } else {
426          exists = !keyIsAfterLast
427              && bloomFilter.contains(key, 0, key.length, bloom);
428        }
429
430        return exists;
431      }
432    } catch (IOException e) {
433      LOG.error("Error reading bloom filter data -- proceeding without",
434          e);
435      setGeneralBloomFilterFaulty();
436    } catch (IllegalArgumentException e) {
437      LOG.error("Bad bloom filter data -- proceeding without", e);
438      setGeneralBloomFilterFaulty();
439    } finally {
440      // Return the bloom block so that its ref count can be decremented.
441      reader.returnBlock(bloomBlock);
442    }
443    return true;
444  }
445
446  /**
447   * Checks whether the given scan rowkey range overlaps with the current storefile's
448   * @param scan the scan specification. Used to determine the rowkey range.
449   * @return true if there is overlap, false otherwise
450   */
451  public boolean passesKeyRangeFilter(Scan scan) {
452    Optional<Cell> firstKeyKV = this.getFirstKey();
453    Optional<Cell> lastKeyKV = this.getLastKey();
454    if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {
455      // the file is empty
456      return false;
457    }
458    if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) &&
459        Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
460      return true;
461    }
462    byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();
463    byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();
464    boolean nonOverLapping = (getComparator()
465        .compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0 &&
466        !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
467          HConstants.EMPTY_END_ROW)) ||
468        getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0,
469          smallestScanRow.length) < 0;
470    return !nonOverLapping;
471  }
472
473  public Map<byte[], byte[]> loadFileInfo() throws IOException {
474    Map<byte [], byte []> fi = reader.loadFileInfo();
475
476    byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
477    if (b != null) {
478      bloomFilterType = BloomType.valueOf(Bytes.toString(b));
479    }
480
481    lastBloomKey = fi.get(LAST_BLOOM_KEY);
482    if(bloomFilterType == BloomType.ROWCOL) {
483      lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
484    }
485    byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
486    if (cnt != null) {
487      deleteFamilyCnt = Bytes.toLong(cnt);
488    }
489
490    return fi;
491  }
492
493  public void loadBloomfilter() {
494    this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
495    this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
496  }
497
498  public void loadBloomfilter(BlockType blockType) {
499    try {
500      if (blockType == BlockType.GENERAL_BLOOM_META) {
501        if (this.generalBloomFilter != null)
502          return; // Bloom has been loaded
503
504        DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
505        if (bloomMeta != null) {
506          // sanity check for NONE Bloom filter
507          if (bloomFilterType == BloomType.NONE) {
508            throw new IOException(
509                "valid bloom filter type not found in FileInfo");
510          } else {
511            generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
512                reader);
513            if (LOG.isTraceEnabled()) {
514              LOG.trace("Loaded " + bloomFilterType.toString() + " "
515                + generalBloomFilter.getClass().getSimpleName()
516                + " metadata for " + reader.getName());
517            }
518          }
519        }
520      } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
521        if (this.deleteFamilyBloomFilter != null)
522          return; // Bloom has been loaded
523
524        DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
525        if (bloomMeta != null) {
526          deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
527              bloomMeta, reader);
528          LOG.info("Loaded Delete Family Bloom ("
529              + deleteFamilyBloomFilter.getClass().getSimpleName()
530              + ") metadata for " + reader.getName());
531        }
532      } else {
533        throw new RuntimeException("Block Type: " + blockType.toString()
534            + "is not supported for Bloom filter");
535      }
536    } catch (IOException e) {
537      LOG.error("Error reading bloom filter meta for " + blockType
538          + " -- proceeding without", e);
539      setBloomFilterFaulty(blockType);
540    } catch (IllegalArgumentException e) {
541      LOG.error("Bad bloom filter meta " + blockType
542          + " -- proceeding without", e);
543      setBloomFilterFaulty(blockType);
544    }
545  }
546
547  private void setBloomFilterFaulty(BlockType blockType) {
548    if (blockType == BlockType.GENERAL_BLOOM_META) {
549      setGeneralBloomFilterFaulty();
550    } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
551      setDeleteFamilyBloomFilterFaulty();
552    }
553  }
554
555  /**
556   * The number of Bloom filter entries in this store file, or an estimate
557   * thereof, if the Bloom filter is not loaded. This always returns an upper
558   * bound of the number of Bloom filter entries.
559   *
560   * @return an estimate of the number of Bloom filter entries in this file
561   */
562  public long getFilterEntries() {
563    return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
564        : reader.getEntries();
565  }
566
567  public void setGeneralBloomFilterFaulty() {
568    generalBloomFilter = null;
569  }
570
571  public void setDeleteFamilyBloomFilterFaulty() {
572    this.deleteFamilyBloomFilter = null;
573  }
574
575  public Optional<Cell> getLastKey() {
576    return reader.getLastKey();
577  }
578
579  public Optional<byte[]> getLastRowKey() {
580    return reader.getLastRowKey();
581  }
582
583  public Optional<Cell> midKey() throws IOException {
584    return reader.midKey();
585  }
586
587  public long length() {
588    return reader.length();
589  }
590
591  public long getTotalUncompressedBytes() {
592    return reader.getTrailer().getTotalUncompressedBytes();
593  }
594
595  public long getEntries() {
596    return reader.getEntries();
597  }
598
599  public long getDeleteFamilyCnt() {
600    return deleteFamilyCnt;
601  }
602
603  public Optional<Cell> getFirstKey() {
604    return reader.getFirstKey();
605  }
606
607  public long indexSize() {
608    return reader.indexSize();
609  }
610
611  public BloomType getBloomFilterType() {
612    return this.bloomFilterType;
613  }
614
615  public long getSequenceID() {
616    return sequenceID;
617  }
618
619  public void setSequenceID(long sequenceID) {
620    this.sequenceID = sequenceID;
621  }
622
623  public void setBulkLoaded(boolean bulkLoadResult) {
624    this.bulkLoadResult = bulkLoadResult;
625  }
626
627  public boolean isBulkLoaded() {
628    return this.bulkLoadResult;
629  }
630
631  BloomFilter getGeneralBloomFilter() {
632    return generalBloomFilter;
633  }
634
635  long getUncompressedDataIndexSize() {
636    return reader.getTrailer().getUncompressedDataIndexSize();
637  }
638
639  public long getTotalBloomSize() {
640    if (generalBloomFilter == null)
641      return 0;
642    return generalBloomFilter.getByteSize();
643  }
644
645  public int getHFileVersion() {
646    return reader.getTrailer().getMajorVersion();
647  }
648
649  public int getHFileMinorVersion() {
650    return reader.getTrailer().getMinorVersion();
651  }
652
653  public HFile.Reader getHFileReader() {
654    return reader;
655  }
656
657  void disableBloomFilterForTesting() {
658    generalBloomFilter = null;
659    this.deleteFamilyBloomFilter = null;
660  }
661
662  public long getMaxTimestamp() {
663    return timeRange == null ? TimeRange.INITIAL_MAX_TIMESTAMP: timeRange.getMax();
664  }
665
666  boolean isSkipResetSeqId() {
667    return skipResetSeqId;
668  }
669
670  void setSkipResetSeqId(boolean skipResetSeqId) {
671    this.skipResetSeqId = skipResetSeqId;
672  }
673
674  public void setListener(Listener listener) {
675    this.listener = listener;
676  }
677
678  public interface Listener {
679    void storeFileReaderClosed(StoreFileReader reader);
680  }
681}