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