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          Cell kvKey = PrivateCellUtil.createFirstOnRow(row, HConstants.EMPTY_BYTE_ARRAY, column);
241          return passesGeneralRowColBloomFilter(kvKey);
242        }
243
244        // For multi-column queries the Bloom filter is checked from the
245        // seekExact operation.
246        return true;
247      case ROWPREFIX_FIXED_LENGTH:
248        return passesGeneralRowPrefixBloomFilter(scan);
249      default:
250        if (scan.isGetScan()) {
251          bloomFilterMetrics.incrementEligible();
252        }
253        return true;
254    }
255  }
256
257  public boolean passesDeleteFamilyBloomFilter(byte[] row, int rowOffset, int rowLen) {
258    // Cache Bloom filter as a local variable in case it is set to null by
259    // another thread on an IO error.
260    BloomFilter bloomFilter = this.deleteFamilyBloomFilter;
261
262    // Empty file or there is no delete family at all
263    if (reader.getTrailer().getEntryCount() == 0 || deleteFamilyCnt == 0) {
264      return false;
265    }
266
267    if (bloomFilter == null) {
268      return true;
269    }
270
271    try {
272      if (!bloomFilter.supportsAutoLoading()) {
273        return true;
274      }
275      return bloomFilter.contains(row, rowOffset, rowLen, null);
276    } catch (IllegalArgumentException e) {
277      LOG.error("Bad Delete Family bloom filter data -- proceeding without", e);
278      setDeleteFamilyBloomFilterFaulty();
279    }
280
281    return true;
282  }
283
284  /**
285   * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a
286   * multi-column query.
287   * @return True if passes
288   */
289  private boolean passesGeneralRowBloomFilter(byte[] row, int rowOffset, int rowLen) {
290    BloomFilter bloomFilter = this.generalBloomFilter;
291    if (bloomFilter == null) {
292      bloomFilterMetrics.incrementEligible();
293      return true;
294    }
295
296    // Used in ROW bloom
297    byte[] key = null;
298    if (rowOffset != 0 || rowLen != row.length) {
299      throw new AssertionError("For row-only Bloom filters the row must occupy the whole array");
300    }
301    key = row;
302    return checkGeneralBloomFilter(key, null, bloomFilter);
303  }
304
305  /**
306   * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a
307   * multi-column query. the cell to check if present in BloomFilter
308   * @return True if passes
309   */
310  public boolean passesGeneralRowColBloomFilter(Cell cell) {
311    BloomFilter bloomFilter = this.generalBloomFilter;
312    if (bloomFilter == null) {
313      bloomFilterMetrics.incrementEligible();
314      return true;
315    }
316    // Used in ROW_COL bloom
317    Cell kvKey = null;
318    // Already if the incoming key is a fake rowcol key then use it as it is
319    if (cell.getTypeByte() == KeyValue.Type.Maximum.getCode() && cell.getFamilyLength() == 0) {
320      kvKey = cell;
321    } else {
322      kvKey = PrivateCellUtil.createFirstOnRowCol(cell);
323    }
324    return checkGeneralBloomFilter(null, kvKey, bloomFilter);
325  }
326
327  /**
328   * A method for checking Bloom filters. Called directly from StoreFileScanner in case of a
329   * multi-column query.
330   * @return True if passes
331   */
332  private boolean passesGeneralRowPrefixBloomFilter(Scan scan) {
333    BloomFilter bloomFilter = this.generalBloomFilter;
334    if (bloomFilter == null) {
335      bloomFilterMetrics.incrementEligible();
336      return true;
337    }
338
339    byte[] row = scan.getStartRow();
340    byte[] rowPrefix;
341    if (scan.isGetScan()) {
342      rowPrefix = Bytes.copy(row, 0, Math.min(prefixLength, row.length));
343    } else {
344      // For non-get scans
345      // Find out the common prefix of startRow and stopRow.
346      int commonLength = Bytes.findCommonPrefix(scan.getStartRow(), scan.getStopRow(),
347        scan.getStartRow().length, scan.getStopRow().length, 0, 0);
348      // startRow and stopRow don't have the common prefix.
349      // Or the common prefix length is less than prefixLength
350      if (commonLength <= 0 || commonLength < prefixLength) {
351        return true;
352      }
353      rowPrefix = Bytes.copy(row, 0, prefixLength);
354    }
355    return checkGeneralBloomFilter(rowPrefix, null, bloomFilter);
356  }
357
358  private boolean checkGeneralBloomFilter(byte[] key, Cell kvKey, BloomFilter bloomFilter) {
359    // Empty file
360    if (reader.getTrailer().getEntryCount() == 0) {
361      return false;
362    }
363    HFileBlock bloomBlock = null;
364    try {
365      boolean shouldCheckBloom;
366      ByteBuff bloom;
367      if (bloomFilter.supportsAutoLoading()) {
368        bloom = null;
369        shouldCheckBloom = true;
370      } else {
371        bloomBlock = reader.getMetaBlock(HFile.BLOOM_FILTER_DATA_KEY, true);
372        bloom = bloomBlock.getBufferWithoutHeader();
373        shouldCheckBloom = bloom != null;
374      }
375
376      if (shouldCheckBloom) {
377        boolean exists;
378
379        // Whether the primary Bloom key is greater than the last Bloom key
380        // from the file info. For row-column Bloom filters this is not yet
381        // a sufficient condition to return false.
382        boolean keyIsAfterLast = (lastBloomKey != null);
383        // hbase:meta does not have blooms. So we need not have special interpretation
384        // of the hbase:meta cells. We can safely use Bytes.BYTES_RAWCOMPARATOR for ROW Bloom
385        if (keyIsAfterLast) {
386          if (bloomFilterType == BloomType.ROWCOL) {
387            keyIsAfterLast = (getComparator().compare(kvKey, lastBloomKeyOnlyKV)) > 0;
388          } else {
389            keyIsAfterLast = (Bytes.BYTES_RAWCOMPARATOR.compare(key, lastBloomKey) > 0);
390          }
391        }
392
393        if (bloomFilterType == BloomType.ROWCOL) {
394          // Since a Row Delete is essentially a DeleteFamily applied to all
395          // columns, a file might be skipped if using row+col Bloom filter.
396          // In order to ensure this file is included an additional check is
397          // required looking only for a row bloom.
398          Cell rowBloomKey = PrivateCellUtil.createFirstOnRow(kvKey);
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 && (getComparator().compare(rowBloomKey, lastBloomKeyOnlyKV)) > 0) {
402            exists = false;
403          } else {
404            exists = bloomFilter.contains(kvKey, bloom, BloomType.ROWCOL)
405              || bloomFilter.contains(rowBloomKey, bloom, BloomType.ROWCOL);
406          }
407        } else {
408          exists = !keyIsAfterLast && bloomFilter.contains(key, 0, key.length, bloom);
409        }
410
411        return exists;
412      }
413    } catch (IOException e) {
414      LOG.error("Error reading bloom filter data -- proceeding without", e);
415      setGeneralBloomFilterFaulty();
416    } catch (IllegalArgumentException e) {
417      LOG.error("Bad bloom filter data -- proceeding without", e);
418      setGeneralBloomFilterFaulty();
419    } finally {
420      // Release the bloom block so that its ref count can be decremented.
421      if (bloomBlock != null) {
422        bloomBlock.release();
423      }
424    }
425    return true;
426  }
427
428  /**
429   * Checks whether the given scan rowkey range overlaps with the current storefile's
430   * @param scan the scan specification. Used to determine the rowkey range.
431   * @return true if there is overlap, false otherwise
432   */
433  public boolean passesKeyRangeFilter(Scan scan) {
434    Optional<ExtendedCell> firstKeyKV = this.getFirstKey();
435    Optional<ExtendedCell> lastKeyKV = this.getLastKey();
436    if (!firstKeyKV.isPresent() || !lastKeyKV.isPresent()) {
437      // the file is empty
438      return false;
439    }
440    if (
441      Bytes.equals(scan.getStartRow(), HConstants.EMPTY_START_ROW)
442        && Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)
443    ) {
444      return true;
445    }
446    byte[] smallestScanRow = scan.isReversed() ? scan.getStopRow() : scan.getStartRow();
447    byte[] largestScanRow = scan.isReversed() ? scan.getStartRow() : scan.getStopRow();
448    boolean nonOverLapping =
449      (getComparator().compareRows(firstKeyKV.get(), largestScanRow, 0, largestScanRow.length) > 0
450        && !Bytes.equals(scan.isReversed() ? scan.getStartRow() : scan.getStopRow(),
451          HConstants.EMPTY_END_ROW))
452        || getComparator().compareRows(lastKeyKV.get(), smallestScanRow, 0, smallestScanRow.length)
453            < 0;
454    return !nonOverLapping;
455  }
456
457  public Map<byte[], byte[]> loadFileInfo() throws IOException {
458    Map<byte[], byte[]> fi = reader.getHFileInfo();
459
460    byte[] b = fi.get(BLOOM_FILTER_TYPE_KEY);
461    if (b != null) {
462      bloomFilterType = BloomType.valueOf(Bytes.toString(b));
463    }
464
465    byte[] p = fi.get(BLOOM_FILTER_PARAM_KEY);
466    if (bloomFilterType == BloomType.ROWPREFIX_FIXED_LENGTH) {
467      prefixLength = Bytes.toInt(p);
468    }
469
470    lastBloomKey = fi.get(LAST_BLOOM_KEY);
471    if (bloomFilterType == BloomType.ROWCOL) {
472      lastBloomKeyOnlyKV = new KeyValue.KeyOnlyKeyValue(lastBloomKey, 0, lastBloomKey.length);
473    }
474    byte[] cnt = fi.get(DELETE_FAMILY_COUNT);
475    if (cnt != null) {
476      deleteFamilyCnt = Bytes.toLong(cnt);
477    }
478
479    return fi;
480  }
481
482  @RestrictedApi(explanation = "Should only be called in tests", link = "",
483      allowedOnPath = ".*/src/test/.*")
484  void loadBloomfilter() {
485    this.loadBloomfilter(BlockType.GENERAL_BLOOM_META, null);
486    this.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, null);
487  }
488
489  public void loadBloomfilter(BlockType blockType, BloomFilterMetrics metrics) {
490    try {
491      this.bloomFilterMetrics = metrics;
492      if (blockType == BlockType.GENERAL_BLOOM_META) {
493        if (this.generalBloomFilter != null) return; // Bloom has been loaded
494
495        DataInput bloomMeta = reader.getGeneralBloomFilterMetadata();
496        if (bloomMeta != null) {
497          // sanity check for NONE Bloom filter
498          if (bloomFilterType == BloomType.NONE) {
499            throw new IOException("valid bloom filter type not found in FileInfo");
500          } else {
501            generalBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, metrics);
502            if (LOG.isTraceEnabled()) {
503              LOG.trace("Loaded " + bloomFilterType.toString() + " "
504                + generalBloomFilter.getClass().getSimpleName() + " metadata for "
505                + reader.getName());
506            }
507          }
508        }
509      } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
510        if (this.deleteFamilyBloomFilter != null) return; // Bloom has been loaded
511
512        DataInput bloomMeta = reader.getDeleteBloomFilterMetadata();
513        if (bloomMeta != null) {
514          // don't pass in metrics for the delete family bloom for now since the
515          // goal is to give users insight into blooms _they_ configured.
516          deleteFamilyBloomFilter = BloomFilterFactory.createFromMeta(bloomMeta, reader, null);
517          LOG.info(
518            "Loaded Delete Family Bloom (" + deleteFamilyBloomFilter.getClass().getSimpleName()
519              + ") metadata for " + reader.getName());
520        }
521      } else {
522        throw new RuntimeException(
523          "Block Type: " + blockType.toString() + "is not supported for Bloom filter");
524      }
525    } catch (IOException e) {
526      LOG.error("Error reading bloom filter meta for " + blockType + " -- proceeding without", e);
527      setBloomFilterFaulty(blockType);
528    } catch (IllegalArgumentException e) {
529      LOG.error("Bad bloom filter meta " + blockType + " -- proceeding without", e);
530      setBloomFilterFaulty(blockType);
531    }
532  }
533
534  @RestrictedApi(explanation = "Should only be called in tests", link = "",
535      allowedOnPath = ".*/StoreFileReader.java|.*/src/test/.*")
536  void setBloomFilterFaulty(BlockType blockType) {
537    if (blockType == BlockType.GENERAL_BLOOM_META) {
538      setGeneralBloomFilterFaulty();
539    } else if (blockType == BlockType.DELETE_FAMILY_BLOOM_META) {
540      setDeleteFamilyBloomFilterFaulty();
541    }
542  }
543
544  /**
545   * The number of Bloom filter entries in this store file, or an estimate thereof, if the Bloom
546   * filter is not loaded. This always returns an upper bound of the number of Bloom filter entries.
547   * @return an estimate of the number of Bloom filter entries in this file
548   */
549  public long getFilterEntries() {
550    return generalBloomFilter != null ? generalBloomFilter.getKeyCount() : reader.getEntries();
551  }
552
553  private void setGeneralBloomFilterFaulty() {
554    generalBloomFilter = null;
555  }
556
557  private void setDeleteFamilyBloomFilterFaulty() {
558    this.deleteFamilyBloomFilter = null;
559  }
560
561  public Optional<ExtendedCell> getLastKey() {
562    return reader.getLastKey();
563  }
564
565  public Optional<byte[]> getLastRowKey() {
566    return reader.getLastRowKey();
567  }
568
569  public Optional<ExtendedCell> midKey() throws IOException {
570    return reader.midKey();
571  }
572
573  public long length() {
574    return reader.length();
575  }
576
577  public long getTotalUncompressedBytes() {
578    return reader.getTrailer().getTotalUncompressedBytes();
579  }
580
581  public long getEntries() {
582    return reader.getEntries();
583  }
584
585  public long getDeleteFamilyCnt() {
586    return deleteFamilyCnt;
587  }
588
589  public Optional<ExtendedCell> getFirstKey() {
590    return reader.getFirstKey();
591  }
592
593  public long indexSize() {
594    return reader.indexSize();
595  }
596
597  public BloomType getBloomFilterType() {
598    return this.bloomFilterType;
599  }
600
601  public long getSequenceID() {
602    return sequenceID;
603  }
604
605  public void setSequenceID(long sequenceID) {
606    this.sequenceID = sequenceID;
607  }
608
609  public void setBulkLoaded(boolean bulkLoadResult) {
610    this.bulkLoadResult = bulkLoadResult;
611  }
612
613  public boolean isBulkLoaded() {
614    return this.bulkLoadResult;
615  }
616
617  BloomFilter getGeneralBloomFilter() {
618    return generalBloomFilter;
619  }
620
621  long getUncompressedDataIndexSize() {
622    return reader.getTrailer().getUncompressedDataIndexSize();
623  }
624
625  public long getTotalBloomSize() {
626    if (generalBloomFilter == null) return 0;
627    return generalBloomFilter.getByteSize();
628  }
629
630  public int getHFileVersion() {
631    return reader.getTrailer().getMajorVersion();
632  }
633
634  public int getHFileMinorVersion() {
635    return reader.getTrailer().getMinorVersion();
636  }
637
638  public HFile.Reader getHFileReader() {
639    return reader;
640  }
641
642  void disableBloomFilterForTesting() {
643    generalBloomFilter = null;
644    this.deleteFamilyBloomFilter = null;
645  }
646
647  public long getMaxTimestamp() {
648    return timeRange == null ? TimeRange.INITIAL_MAX_TIMESTAMP : timeRange.getMax();
649  }
650
651  boolean isSkipResetSeqId() {
652    return skipResetSeqId;
653  }
654
655  void setSkipResetSeqId(boolean skipResetSeqId) {
656    this.skipResetSeqId = skipResetSeqId;
657  }
658
659  public int getPrefixLength() {
660    return prefixLength;
661  }
662
663  public ReaderContext getReaderContext() {
664    return this.context;
665  }
666}