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