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}