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