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