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