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