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}