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 Do not write further code which depends on this call. Instead
195   *   use getStoreFileScanner() which uses the StoreFileScanner class/interface
196   *   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   */
202  @Deprecated
203  public HFileScanner getScanner(boolean cacheBlocks, boolean pread) {
204    return getScanner(cacheBlocks, pread, false);
205  }
206
207  /**
208   * @deprecated Do not write further code which depends on this call. Instead
209   *   use getStoreFileScanner() which uses the StoreFileScanner class/interface
210   *   which is the preferred way to scan a store with higher level concepts.
211   *
212   * @param cacheBlocks
213   *          should we cache the blocks?
214   * @param pread
215   *          use pread (for concurrent small readers)
216   * @param isCompaction
217   *          is scanner being used for compaction?
218   * @return the underlying HFileScanner
219   */
220  @Deprecated
221  public HFileScanner getScanner(boolean cacheBlocks, boolean pread,
222      boolean isCompaction) {
223    return reader.getScanner(cacheBlocks, pread, isCompaction);
224  }
225
226  public void close(boolean evictOnClose) throws IOException {
227    synchronized (this) {
228      if (closed) {
229        return;
230      }
231      reader.close(evictOnClose);
232      closed = true;
233    }
234    if (listener != null) {
235      listener.storeFileReaderClosed(this);
236    }
237  }
238
239  /**
240   * Check if this storeFile may contain keys within the TimeRange that
241   * have not expired (i.e. not older than oldestUnexpiredTS).
242   * @param timeRange the timeRange to restrict
243   * @param oldestUnexpiredTS the oldest timestamp that is not expired, as
244   *          determined by the column family's TTL
245   * @return false if queried keys definitely don't exist in this StoreFile
246   */
247  boolean passesTimerangeFilter(TimeRange tr, long oldestUnexpiredTS) {
248    return this.timeRange == null? true:
249      this.timeRange.includesTimeRange(tr) && this.timeRange.getMax() >= oldestUnexpiredTS;
250  }
251
252  /**
253   * Checks whether the given scan passes the Bloom filter (if present). Only
254   * checks Bloom filters for single-row or single-row-column scans. Bloom
255   * filter checking for multi-gets is implemented as part of the store
256   * scanner system (see {@link StoreFileScanner#seek(Cell)} and uses
257   * the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
258   * and {@link #passesGeneralRowColBloomFilter(Cell)}.
259   *
260   * @param scan the scan specification. Used to determine the row, and to
261   *          check whether this is a single-row ("get") scan.
262   * @param columns the set of columns. Only used for row-column Bloom
263   *          filters.
264   * @return true if the scan with the given column set passes the Bloom
265   *         filter, or if the Bloom filter is not applicable for the scan.
266   *         False if the Bloom filter is applicable and the scan fails it.
267   */
268  boolean passesBloomFilter(Scan scan, final SortedSet<byte[]> columns) {
269    // Multi-column non-get scans will use Bloom filters through the
270    // lower-level API function that this function calls.
271    if (!scan.isGetScan()) {
272      return true;
273    }
274
275    byte[] row = scan.getStartRow();
276    switch (this.bloomFilterType) {
277      case ROW:
278        return passesGeneralRowBloomFilter(row, 0, row.length);
279
280      case ROWCOL:
281        if (columns != null && columns.size() == 1) {
282          byte[] column = columns.first();
283          // create the required fake key
284          Cell kvKey = PrivateCellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
285          return passesGeneralRowColBloomFilter(kvKey);
286        }
287
288        // For multi-column queries the Bloom filter is checked from the
289        // seekExact operation.
290        return true;
291
292      default:
293        return true;
294    }
295  }
296
297  public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset,
298      int rowLen) {
299    // Cache Bloom filter as a local variable in case it is set to null by
300    // another thread on an IO error.
301    BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
302
303    // Empty file or there is no delete family at all
304    if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
305      return false;
306    }
307
308    if (bloomFilter == null) {
309      return true;
310    }
311
312    try {
313      if (!bloomFilter.supportsAutoLoading()) {
314        return true;
315      }
316      return bloomFilter.contains(row, rowOffset, rowLen, null);
317    } catch (IllegalArgumentException e) {
318      LOG.error("Bad Delete Family bloom filter data -- proceeding without",
319          e);
320      setDeleteFamilyBloomFilterFaulty();
321    }
322
323    return true;
324  }
325
326  /**
327   * A method for checking Bloom filters. Called directly from
328   * StoreFileScanner in case of a multi-column query.
329   *
330   * @return True if passes
331   */
332  public boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
333    BloomFilter bloomFilter = this.generalBloomFilter;
334    if (bloomFilter == null) {
335      return true;
336    }
337
338    // Used in ROW bloom
339    byte[] key = null;
340    if (rowOffset != 0 || rowLen != row.length) {
341      throw new AssertionError(
342          "For row-only Bloom filters the row " + "must occupy the whole array");
343    }
344    key = row;
345    return checkGeneralBloomFilter(key, null, bloomFilter);
346  }
347
348  /**
349   * A method for checking Bloom filters. Called directly from
350   * StoreFileScanner in case of a multi-column query.
351   *
352   * @param cell
353   *          the cell to check if present in BloomFilter
354   * @return True if passes
355   */
356  public boolean passesGeneralRowColBloomFilter(Cell cell) {
357    BloomFilter bloomFilter = this.generalBloomFilter;
358    if (bloomFilter == null) {
359      return true;
360    }
361    // Used in ROW_COL bloom
362    Cell kvKey = null;
363    // Already if the incoming key is a fake rowcol key then use it as it is
364    if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
365      kvKey = cell;
366    } else {
367      kvKey = PrivateCellUtil.createFirstOnRowCol(cell);
368    }
369    return checkGeneralBloomFilter(null, kvKey, bloomFilter);
370  }
371
372  private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
373    // Empty file
374    if (reader.getTrailer().getEntryCount() == 0) {
375      return false;
376    }
377    HFileBlock bloomBlock = null;
378    try {
379      boolean shouldCheckBloom;
380      ByteBuff bloom;
381      if (bloomFilter.supportsAutoLoading()) {
382        bloom = null;
383        shouldCheckBloom = true;
384      } else {
385        bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
386        bloom = bloomBlock.getBufferWithoutHeader();
387        shouldCheckBloom = bloom != null;
388      }
389
390      if (shouldCheckBloom) {
391        boolean exists;
392
393        // Whether the primary Bloom key is greater than the last Bloom key
394        // from the file info. For row-column Bloom filters this is not yet
395        // a sufficient condition to return false.
396        boolean keyIsAfterLast = (lastBloomKey != null);
397        // hbase:meta does not have blooms. So we need not have special interpretation
398        // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
399        if (keyIsAfterLast) {
400          if (bloomFilterType == BloomType.ROW) {
401            keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
402          } else {
403            keyIsAfterLast = (CellComparator.getInstance().compare(kvKey, lastBloomKeyOnlyKV)) > 0;
404          }
405        }
406
407        if (bloomFilterType == BloomType.ROWCOL) {
408          // Since a Row Delete is essentially a DeleteFamily applied to all
409          // columns, a file might be skipped if using row+col Bloom filter.
410          // In order to ensure this file is included an additional check is
411          // required looking only for a row bloom.
412          Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);
413          // hbase:meta does not have blooms. So we need not have special interpretation
414          // of the hbase:meta cells.  We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
415          if (keyIsAfterLast
416              && (CellComparator.getInstance().compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
417            exists = false;
418          } else {
419            exists =
420                bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL) ||
421                bloomFilter.contains(rowBloomKey, bloom, BloomType.ROWCOL);
422          }
423        } else {
424          exists = !keyIsAfterLast
425              && bloomFilter.contains(key, 0, key.length, bloom);
426        }
427
428        return exists;
429      }
430    } catch (IOException e) {
431      LOG.error("Error reading bloom filter data -- proceeding without",
432          e);
433      setGeneralBloomFilterFaulty();
434    } catch (IllegalArgumentException e) {
435      LOG.error("Bad bloom filter data -- proceeding without", e);
436      setGeneralBloomFilterFaulty();
437    } finally {
438      // Return the bloom block so that its ref count can be decremented.
439      reader.returnBlock(bloomBlock);
440    }
441    return true;
442  }
443
444  /**
445   * Checks whether the given scan rowkey range overlaps with the current storefile's
446   * @param scan the scan specification. Used to determine the rowkey range.
447   * @return true if there is overlap, false otherwise
448   */
449  public boolean passesKeyRangeFilter(Scan scan) {
450    Optional<Cell> firstKeyKV = this.getFirstKey();
451    Optional<Cell> lastKeyKV = this.getLastKey();
452    if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {
453      // the file is empty
454      return false;
455    }
456    if (Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW) &&
457        Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
458      return true;
459    }
460    byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();
461    byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();
462    boolean nonOverLapping = (getComparator()
463        .compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0 &&
464        !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
465          HConstants.EMPTY_END_ROW)) ||
466        getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0,
467          smallestScanRow.length) < 0;
468    return !nonOverLapping;
469  }
470
471  public Map<byte[], byte[]> loadFileInfo() throws IOException {
472    Map<byte [], byte []> fi = reader.loadFileInfo();
473
474    byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
475    if (b != null) {
476      bloomFilterType = BloomType.valueOf(Bytes.toString(b));
477    }
478
479    lastBloomKey = fi.get(LAST_BLOOM_KEY);
480    if(bloomFilterType == BloomType.ROWCOL) {
481      lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
482    }
483    byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
484    if (cnt != null) {
485      deleteFamilyCnt = Bytes.toLong(cnt);
486    }
487
488    return fi;
489  }
490
491  public void loadBloomfilter() {
492    this.loadBloomfilter(BlockType.GENERAL_BLOOM_META);
493    this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META);
494  }
495
496  public void loadBloomfilter(BlockType blockType) {
497    try {
498      if (blockType == BlockType.GENERAL_BLOOM_META) {
499        if (this.generalBloomFilter != null)
500          return; // Bloom has been loaded
501
502        DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
503        if (bloomMeta != null) {
504          // sanity check for NONE Bloom filter
505          if (bloomFilterType == BloomType.NONE) {
506            throw new IOException(
507                "valid bloom filter type not found in FileInfo");
508          } else {
509            generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta,
510                reader);
511            if (LOG.isTraceEnabled()) {
512              LOG.trace("Loaded " + bloomFilterType.toString() + " "
513                + generalBloomFilter.getClass().getSimpleName()
514                + " metadata for " + reader.getName());
515            }
516          }
517        }
518      } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
519        if (this.deleteFamilyBloomFilter != null)
520          return; // Bloom has been loaded
521
522        DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
523        if (bloomMeta != null) {
524          deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(
525              bloomMeta, reader);
526          LOG.info("Loaded Delete Family Bloom ("
527              + deleteFamilyBloomFilter.getClass().getSimpleName()
528              + ") metadata for " + reader.getName());
529        }
530      } else {
531        throw new RuntimeException("Block Type: " + blockType.toString()
532            + "is not supported for Bloom filter");
533      }
534    } catch (IOException e) {
535      LOG.error("Error reading bloom filter meta for " + blockType
536          + " -- proceeding without", e);
537      setBloomFilterFaulty(blockType);
538    } catch (IllegalArgumentException e) {
539      LOG.error("Bad bloom filter meta " + blockType
540          + " -- proceeding without", e);
541      setBloomFilterFaulty(blockType);
542    }
543  }
544
545  private void setBloomFilterFaulty(BlockType blockType) {
546    if (blockType == BlockType.GENERAL_BLOOM_META) {
547      setGeneralBloomFilterFaulty();
548    } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
549      setDeleteFamilyBloomFilterFaulty();
550    }
551  }
552
553  /**
554   * The number of Bloom filter entries in this store file, or an estimate
555   * thereof, if the Bloom filter is not loaded. This always returns an upper
556   * bound of the number of Bloom filter entries.
557   *
558   * @return an estimate of the number of Bloom filter entries in this file
559   */
560  public long getFilterEntries() {
561    return generalBloomFilter != null ? generalBloomFilter.getKeyCount()
562        : reader.getEntries();
563  }
564
565  public void setGeneralBloomFilterFaulty() {
566    generalBloomFilter = null;
567  }
568
569  public void setDeleteFamilyBloomFilterFaulty() {
570    this.deleteFamilyBloomFilter = null;
571  }
572
573  public Optional<Cell> getLastKey() {
574    return reader.getLastKey();
575  }
576
577  public Optional<byte[]> getLastRowKey() {
578    return reader.getLastRowKey();
579  }
580
581  public Optional<Cell> midKey() throws IOException {
582    return reader.midKey();
583  }
584
585  public long length() {
586    return reader.length();
587  }
588
589  public long getTotalUncompressedBytes() {
590    return reader.getTrailer().getTotalUncompressedBytes();
591  }
592
593  public long getEntries() {
594    return reader.getEntries();
595  }
596
597  public long getDeleteFamilyCnt() {
598    return deleteFamilyCnt;
599  }
600
601  public Optional<Cell> getFirstKey() {
602    return reader.getFirstKey();
603  }
604
605  public long indexSize() {
606    return reader.indexSize();
607  }
608
609  public BloomType getBloomFilterType() {
610    return this.bloomFilterType;
611  }
612
613  public long getSequenceID() {
614    return sequenceID;
615  }
616
617  public void setSequenceID(long sequenceID) {
618    this.sequenceID = sequenceID;
619  }
620
621  public void setBulkLoaded(boolean bulkLoadResult) {
622    this.bulkLoadResult = bulkLoadResult;
623  }
624
625  public boolean isBulkLoaded() {
626    return this.bulkLoadResult;
627  }
628
629  BloomFilter getGeneralBloomFilter() {
630    return generalBloomFilter;
631  }
632
633  long getUncompressedDataIndexSize() {
634    return reader.getTrailer().getUncompressedDataIndexSize();
635  }
636
637  public long getTotalBloomSize() {
638    if (generalBloomFilter == null)
639      return 0;
640    return generalBloomFilter.getByteSize();
641  }
642
643  public int getHFileVersion() {
644    return reader.getTrailer().getMajorVersion();
645  }
646
647  public int getHFileMinorVersion() {
648    return reader.getTrailer().getMinorVersion();
649  }
650
651  public HFile.Reader getHFileReader() {
652    return reader;
653  }
654
655  void disableBloomFilterForTesting() {
656    generalBloomFilter = null;
657    this.deleteFamilyBloomFilter = null;
658  }
659
660  public long getMaxTimestamp() {
661    return timeRange == null ? TimeRange.INITIAL_MAX_TIMESTAMP: timeRange.getMax();
662  }
663
664  boolean isSkipResetSeqId() {
665    return skipResetSeqId;
666  }
667
668  void setSkipResetSeqId(boolean skipResetSeqId) {
669    this.skipResetSeqId = skipResetSeqId;
670  }
671
672  public void setListener(Listener listener) {
673    this.listener = listener;
674  }
675
676  public interface Listener {
677    void storeFileReaderClosed(StoreFileReader reader);
678  }
679}