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