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