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