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