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