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