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 java.io.IOException; 021import java.io.UnsupportedEncodingException; 022import java.net.URLEncoder; 023import java.util.Collection; 024import java.util.Collections; 025import java.util.HashSet; 026import java.util.Map; 027import java.util.Optional; 028import java.util.OptionalLong; 029import java.util.Set; 030import java.util.concurrent.atomic.AtomicBoolean; 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.fs.FSDataInputStream; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.Cell; 036import org.apache.hadoop.hbase.CellComparator; 037import org.apache.hadoop.hbase.HConstants; 038import org.apache.hadoop.hbase.HDFSBlocksDistribution; 039import org.apache.hadoop.hbase.io.TimeRange; 040import org.apache.hadoop.hbase.io.hfile.BlockType; 041import org.apache.hadoop.hbase.io.hfile.BloomFilterMetrics; 042import org.apache.hadoop.hbase.io.hfile.CacheConfig; 043import org.apache.hadoop.hbase.io.hfile.HFile; 044import org.apache.hadoop.hbase.io.hfile.ReaderContext; 045import org.apache.hadoop.hbase.io.hfile.ReaderContext.ReaderType; 046import org.apache.hadoop.hbase.util.BloomFilterFactory; 047import org.apache.hadoop.hbase.util.Bytes; 048import org.apache.yetus.audience.InterfaceAudience; 049import org.slf4j.Logger; 050import org.slf4j.LoggerFactory; 051 052import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; 053 054import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 055 056/** 057 * A Store data file. Stores usually have one or more of these files. They are produced by flushing 058 * the memstore to disk. To create, instantiate a writer using {@link StoreFileWriter.Builder} and 059 * append data. Be sure to add any metadata before calling close on the Writer (Use the 060 * appendMetadata convenience methods). On close, a StoreFile is sitting in the Filesystem. To refer 061 * to it, create a StoreFile instance passing filesystem and path. To read, call 062 * {@link #initReader()} 063 * <p> 064 * StoreFiles may also reference store files in another Store. The reason for this weird pattern 065 * where you use a different instance for the writer and a reader is that we write once but read a 066 * lot more. 067 */ 068@InterfaceAudience.Private 069public class HStoreFile implements StoreFile { 070 071 private static final Logger LOG = LoggerFactory.getLogger(HStoreFile.class.getName()); 072 073 // Keys for fileinfo values in HFile 074 075 /** Max Sequence ID in FileInfo */ 076 public static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY"); 077 078 /** Major compaction flag in FileInfo */ 079 public static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY"); 080 081 /** Minor compaction flag in FileInfo */ 082 public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY = 083 Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION"); 084 085 /** 086 * Key for compaction event which contains the compacted storefiles in FileInfo 087 */ 088 public static final byte[] COMPACTION_EVENT_KEY = Bytes.toBytes("COMPACTION_EVENT_KEY"); 089 090 /** Bloom filter Type in FileInfo */ 091 public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE"); 092 093 /** Bloom filter param in FileInfo */ 094 public static final byte[] BLOOM_FILTER_PARAM_KEY = Bytes.toBytes("BLOOM_FILTER_PARAM"); 095 096 /** Delete Family Count in FileInfo */ 097 public static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT"); 098 099 /** Last Bloom filter key in FileInfo */ 100 public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY"); 101 102 /** Key for Timerange information in metadata */ 103 public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE"); 104 105 /** Key for timestamp of earliest-put in metadata */ 106 public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS"); 107 108 /** Key for the number of mob cells in metadata */ 109 public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT"); 110 111 /** Null data */ 112 public static final byte[] NULL_VALUE = new byte[] { 0 }; 113 114 /** Key for the list of MOB file references */ 115 public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS"); 116 117 /** Meta key set when store file is a result of a bulk load */ 118 public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK"); 119 public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP"); 120 121 /** 122 * Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets 123 * the cell seqId with the latest one, if this metadata is set as true, the reset is skipped. 124 */ 125 public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID"); 126 127 private final StoreFileInfo fileInfo; 128 129 // StoreFile.Reader 130 private volatile StoreFileReader initialReader; 131 private volatile InputStreamBlockDistribution initialReaderBlockDistribution = null; 132 133 // Block cache configuration and reference. 134 private final CacheConfig cacheConf; 135 private final BloomFilterMetrics metrics; 136 137 // Indicates if the file got compacted 138 private volatile boolean compactedAway = false; 139 140 // Keys for metadata stored in backing HFile. 141 // Set when we obtain a Reader. 142 private long sequenceid = -1; 143 144 // max of the MemstoreTS in the KV's in this store 145 // Set when we obtain a Reader. 146 private long maxMemstoreTS = -1; 147 148 // firstKey, lastkey and cellComparator will be set when openReader. 149 private Optional<Cell> firstKey; 150 151 private Optional<Cell> lastKey; 152 153 private CellComparator comparator; 154 155 public CacheConfig getCacheConf() { 156 return this.cacheConf; 157 } 158 159 @Override 160 public Optional<Cell> getFirstKey() { 161 return firstKey; 162 } 163 164 @Override 165 public Optional<Cell> getLastKey() { 166 return lastKey; 167 } 168 169 @Override 170 public CellComparator getComparator() { 171 return comparator; 172 } 173 174 @Override 175 public long getMaxMemStoreTS() { 176 return maxMemstoreTS; 177 } 178 179 // If true, this file was product of a major compaction. Its then set 180 // whenever you get a Reader. 181 private AtomicBoolean majorCompaction = null; 182 183 // If true, this file should not be included in minor compactions. 184 // It's set whenever you get a Reader. 185 private boolean excludeFromMinorCompaction = false; 186 187 // This file was product of these compacted store files 188 private final Set<String> compactedStoreFiles = new HashSet<>(); 189 190 /** 191 * Map of the metadata entries in the corresponding HFile. Populated when Reader is opened after 192 * which it is not modified again. 193 */ 194 private Map<byte[], byte[]> metadataMap; 195 196 /** 197 * Bloom filter type specified in column family configuration. Does not necessarily correspond to 198 * the Bloom filter type present in the HFile. 199 */ 200 private final BloomType cfBloomType; 201 202 /** 203 * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram 204 * depending on the underlying files (10-20MB?). 205 * @param fs The current file system to use. 206 * @param p The path of the file. 207 * @param conf The current configuration. 208 * @param cacheConf The cache configuration and block cache reference. 209 * @param cfBloomType The bloom type to use for this store file as specified by column family 210 * configuration. This may or may not be the same as the Bloom filter type 211 * actually present in the HFile, because column family configuration might 212 * change. If this is {@link BloomType#NONE}, the existing Bloom filter is 213 * ignored. 214 * @param primaryReplica true if this is a store file for primary replica, otherwise false. 215 */ 216 public HStoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf, 217 BloomType cfBloomType, boolean primaryReplica) throws IOException { 218 this(new StoreFileInfo(conf, fs, p, primaryReplica), cfBloomType, cacheConf); 219 } 220 221 /** 222 * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram 223 * depending on the underlying files (10-20MB?). 224 * @param fileInfo The store file information. 225 * @param cfBloomType The bloom type to use for this store file as specified by column family 226 * configuration. This may or may not be the same as the Bloom filter type 227 * actually present in the HFile, because column family configuration might 228 * change. If this is {@link BloomType#NONE}, the existing Bloom filter is 229 * ignored. 230 * @param cacheConf The cache configuration and block cache reference. 231 */ 232 public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf) { 233 this(fileInfo, cfBloomType, cacheConf, null); 234 } 235 236 /** 237 * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram 238 * depending on the underlying files (10-20MB?). 239 * @param fileInfo The store file information. 240 * @param cfBloomType The bloom type to use for this store file as specified by column family 241 * configuration. This may or may not be the same as the Bloom filter type 242 * actually present in the HFile, because column family configuration might 243 * change. If this is {@link BloomType#NONE}, the existing Bloom filter is 244 * ignored. 245 * @param cacheConf The cache configuration and block cache reference. 246 * @param metrics Tracks bloom filter requests and results. May be null. 247 */ 248 public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf, 249 BloomFilterMetrics metrics) { 250 this.fileInfo = fileInfo; 251 this.cacheConf = cacheConf; 252 this.metrics = metrics; 253 if (BloomFilterFactory.isGeneralBloomEnabled(fileInfo.getConf())) { 254 this.cfBloomType = cfBloomType; 255 } else { 256 LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType=" 257 + cfBloomType + " (disabled in config)"); 258 this.cfBloomType = BloomType.NONE; 259 } 260 } 261 262 /** 263 * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a 264 * reference. 265 */ 266 public StoreFileInfo getFileInfo() { 267 return this.fileInfo; 268 } 269 270 @Override 271 public Path getPath() { 272 return this.fileInfo.getPath(); 273 } 274 275 @Override 276 public Path getEncodedPath() { 277 try { 278 return new Path(URLEncoder.encode(fileInfo.getPath().toString(), HConstants.UTF8_ENCODING)); 279 } catch (UnsupportedEncodingException ex) { 280 throw new RuntimeException("URLEncoder doesn't support UTF-8", ex); 281 } 282 } 283 284 @Override 285 public Path getQualifiedPath() { 286 FileSystem fs = fileInfo.getFileSystem(); 287 return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory()); 288 } 289 290 @Override 291 public boolean isReference() { 292 return this.fileInfo.isReference(); 293 } 294 295 @Override 296 public boolean isHFile() { 297 return StoreFileInfo.isHFile(this.fileInfo.getPath()); 298 } 299 300 @Override 301 public boolean isMajorCompactionResult() { 302 if (this.majorCompaction == null) { 303 throw new NullPointerException("This has not been set yet"); 304 } 305 return this.majorCompaction.get(); 306 } 307 308 @Override 309 public boolean excludeFromMinorCompaction() { 310 return this.excludeFromMinorCompaction; 311 } 312 313 @Override 314 public long getMaxSequenceId() { 315 return this.sequenceid; 316 } 317 318 @Override 319 public long getModificationTimeStamp() throws IOException { 320 return getModificationTimestamp(); 321 } 322 323 @Override 324 public long getModificationTimestamp() throws IOException { 325 return fileInfo.getModificationTime(); 326 } 327 328 /** 329 * @param key to look up 330 * @return value associated with the metadata key 331 */ 332 public byte[] getMetadataValue(byte[] key) { 333 return metadataMap.get(key); 334 } 335 336 @Override 337 public boolean isBulkLoadResult() { 338 return StoreFileInfo.hasBulkloadSeqId(this.getPath()) 339 || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY)); 340 } 341 342 public boolean isCompactedAway() { 343 return compactedAway; 344 } 345 346 public int getRefCount() { 347 return fileInfo.getRefCount(); 348 } 349 350 /** Returns true if the file is still used in reads */ 351 public boolean isReferencedInReads() { 352 int rc = fileInfo.getRefCount(); 353 assert rc >= 0; // we should not go negative. 354 return rc > 0; 355 } 356 357 @Override 358 public OptionalLong getBulkLoadTimestamp() { 359 byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY); 360 return bulkLoadTimestamp == null 361 ? OptionalLong.empty() 362 : OptionalLong.of(Bytes.toLong(bulkLoadTimestamp)); 363 } 364 365 /** 366 * @return the cached value of HDFS blocks distribution. The cached value is calculated when store 367 * file is opened. 368 */ 369 public HDFSBlocksDistribution getHDFSBlockDistribution() { 370 if (initialReaderBlockDistribution != null) { 371 return initialReaderBlockDistribution.getHDFSBlockDistribution(); 372 } else { 373 return this.fileInfo.getHDFSBlockDistribution(); 374 } 375 } 376 377 /** 378 * Opens reader on this store file. Called by Constructor. 379 * @see #closeStoreFile(boolean) 380 */ 381 private void open() throws IOException { 382 fileInfo.initHDFSBlocksDistribution(); 383 long readahead = fileInfo.isNoReadahead() ? 0L : -1L; 384 ReaderContext context = fileInfo.createReaderContext(false, readahead, ReaderType.PREAD); 385 fileInfo.initHFileInfo(context); 386 StoreFileReader reader = fileInfo.preStoreFileReaderOpen(context, cacheConf); 387 if (reader == null) { 388 reader = fileInfo.createReader(context, cacheConf); 389 fileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader()); 390 } 391 this.initialReader = fileInfo.postStoreFileReaderOpen(context, cacheConf, reader); 392 393 if (InputStreamBlockDistribution.isEnabled(fileInfo.getConf())) { 394 boolean useHBaseChecksum = context.getInputStreamWrapper().shouldUseHBaseChecksum(); 395 FSDataInputStream stream = context.getInputStreamWrapper().getStream(useHBaseChecksum); 396 this.initialReaderBlockDistribution = new InputStreamBlockDistribution(stream, fileInfo); 397 } 398 399 // Load up indices and fileinfo. This also loads Bloom filter type. 400 metadataMap = Collections.unmodifiableMap(initialReader.loadFileInfo()); 401 402 // Read in our metadata. 403 byte[] b = metadataMap.get(MAX_SEQ_ID_KEY); 404 if (b != null) { 405 // By convention, if halfhfile, top half has a sequence number > bottom 406 // half. Thats why we add one in below. Its done for case the two halves 407 // are ever merged back together --rare. Without it, on open of store, 408 // since store files are distinguished by sequence id, the one half would 409 // subsume the other. 410 this.sequenceid = Bytes.toLong(b); 411 if (fileInfo.isTopReference()) { 412 this.sequenceid += 1; 413 } 414 } 415 416 if (isBulkLoadResult()) { 417 // For bulkloads, we have to parse the sequenceid from the path name 418 OptionalLong sequenceId = StoreFileInfo.getBulkloadSeqId(this.getPath()); 419 if (sequenceId.isPresent()) { 420 this.sequenceid = sequenceId.getAsLong(); 421 // Handle reference files as done above. 422 if (fileInfo.isTopReference()) { 423 this.sequenceid += 1; 424 } 425 } 426 427 // SKIP_RESET_SEQ_ID only works in bulk loaded file. 428 // In mob compaction, the hfile where the cells contain the path of a new mob file is bulk 429 // loaded to hbase, these cells have the same seqIds with the old ones. We do not want 430 // to reset new seqIds for them since this might make a mess of the visibility of cells that 431 // have the same row key but different seqIds. 432 boolean skipResetSeqId = isSkipResetSeqId(metadataMap.get(SKIP_RESET_SEQ_ID)); 433 if (skipResetSeqId) { 434 // increase the seqId when it is a bulk loaded file from mob compaction. 435 this.sequenceid += 1; 436 } 437 initialReader.setSkipResetSeqId(skipResetSeqId); 438 initialReader.setBulkLoaded(true); 439 } 440 initialReader.setSequenceID(this.sequenceid); 441 442 b = metadataMap.get(HFile.Writer.MAX_MEMSTORE_TS_KEY); 443 if (b != null) { 444 this.maxMemstoreTS = Bytes.toLong(b); 445 } 446 447 b = metadataMap.get(MAJOR_COMPACTION_KEY); 448 if (b != null) { 449 boolean mc = Bytes.toBoolean(b); 450 if (this.majorCompaction == null) { 451 this.majorCompaction = new AtomicBoolean(mc); 452 } else { 453 this.majorCompaction.set(mc); 454 } 455 } else { 456 // Presume it is not major compacted if it doesn't explicity say so 457 // HFileOutputFormat explicitly sets the major compacted key. 458 this.majorCompaction = new AtomicBoolean(false); 459 } 460 461 b = metadataMap.get(EXCLUDE_FROM_MINOR_COMPACTION_KEY); 462 this.excludeFromMinorCompaction = (b != null && Bytes.toBoolean(b)); 463 464 BloomType hfileBloomType = initialReader.getBloomFilterType(); 465 if (cfBloomType != BloomType.NONE) { 466 initialReader.loadBloomfilter(BlockType.GENERAL_BLOOM_META, metrics); 467 if (hfileBloomType != cfBloomType) { 468 LOG.debug("HFile Bloom filter type for " + initialReader.getHFileReader().getName() + ": " 469 + hfileBloomType + ", but " + cfBloomType + " specified in column family " 470 + "configuration"); 471 } 472 } else if (hfileBloomType != BloomType.NONE) { 473 LOG.info( 474 "Bloom filter turned off by CF config for " + initialReader.getHFileReader().getName()); 475 } 476 477 // load delete family bloom filter 478 initialReader.loadBloomfilter(BlockType.DELETE_FAMILY_BLOOM_META, metrics); 479 480 try { 481 byte[] data = metadataMap.get(TIMERANGE_KEY); 482 initialReader.timeRange = 483 data == null ? null : TimeRangeTracker.parseFrom(data).toTimeRange(); 484 } catch (IllegalArgumentException e) { 485 LOG.error("Error reading timestamp range data from meta -- " + "proceeding without", e); 486 this.initialReader.timeRange = null; 487 } 488 489 try { 490 byte[] data = metadataMap.get(COMPACTION_EVENT_KEY); 491 this.compactedStoreFiles.addAll(ProtobufUtil.toCompactedStoreFiles(data)); 492 } catch (IOException e) { 493 LOG.error("Error reading compacted storefiles from meta data", e); 494 } 495 496 // initialize so we can reuse them after reader closed. 497 firstKey = initialReader.getFirstKey(); 498 lastKey = initialReader.getLastKey(); 499 comparator = initialReader.getComparator(); 500 } 501 502 /** 503 * Initialize the reader used for pread. 504 */ 505 public void initReader() throws IOException { 506 if (initialReader == null) { 507 synchronized (this) { 508 if (initialReader == null) { 509 try { 510 open(); 511 } catch (Exception e) { 512 try { 513 boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true; 514 this.closeStoreFile(evictOnClose); 515 } catch (IOException ee) { 516 LOG.warn("failed to close reader", ee); 517 } 518 throw e; 519 } 520 } 521 } 522 } 523 } 524 525 private StoreFileReader createStreamReader(boolean canUseDropBehind) throws IOException { 526 initReader(); 527 final boolean doDropBehind = canUseDropBehind && cacheConf.shouldDropBehindCompaction(); 528 ReaderContext context = fileInfo.createReaderContext(doDropBehind, -1, ReaderType.STREAM); 529 StoreFileReader reader = fileInfo.preStoreFileReaderOpen(context, cacheConf); 530 if (reader == null) { 531 reader = fileInfo.createReader(context, cacheConf); 532 // steam reader need copy stuffs from pread reader 533 reader.copyFields(initialReader); 534 } 535 return fileInfo.postStoreFileReaderOpen(context, cacheConf, reader); 536 } 537 538 /** 539 * Get a scanner which uses pread. 540 * <p> 541 * Must be called after initReader. 542 */ 543 public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder, 544 boolean canOptimizeForNonNullColumn) { 545 return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder, 546 canOptimizeForNonNullColumn); 547 } 548 549 /** 550 * Get a scanner which uses streaming read. 551 * <p> 552 * Must be called after initReader. 553 */ 554 public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks, 555 boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) 556 throws IOException { 557 return createStreamReader(canUseDropBehind).getStoreFileScanner(cacheBlocks, false, 558 isCompaction, readPt, scannerOrder, canOptimizeForNonNullColumn); 559 } 560 561 /** 562 * @return Current reader. Must call initReader first else returns null. 563 * @see #initReader() 564 */ 565 public StoreFileReader getReader() { 566 return this.initialReader; 567 } 568 569 /** 570 * @param evictOnClose whether to evict blocks belonging to this file 571 */ 572 public synchronized void closeStoreFile(boolean evictOnClose) throws IOException { 573 if (this.initialReader != null) { 574 this.initialReader.close(evictOnClose); 575 this.initialReader = null; 576 } 577 } 578 579 /** 580 * Delete this file 581 */ 582 public void deleteStoreFile() throws IOException { 583 boolean evictOnClose = cacheConf != null ? cacheConf.shouldEvictOnClose() : true; 584 closeStoreFile(evictOnClose); 585 this.fileInfo.getFileSystem().delete(getPath(), true); 586 } 587 588 public void markCompactedAway() { 589 this.compactedAway = true; 590 } 591 592 @Override 593 public String toString() { 594 return this.fileInfo.toString(); 595 } 596 597 @Override 598 public String toStringDetailed() { 599 StringBuilder sb = new StringBuilder(); 600 sb.append(this.getPath().toString()); 601 sb.append(", isReference=").append(isReference()); 602 sb.append(", isBulkLoadResult=").append(isBulkLoadResult()); 603 if (isBulkLoadResult()) { 604 sb.append(", bulkLoadTS="); 605 OptionalLong bulkLoadTS = getBulkLoadTimestamp(); 606 if (bulkLoadTS.isPresent()) { 607 sb.append(bulkLoadTS.getAsLong()); 608 } else { 609 sb.append("NotPresent"); 610 } 611 } else { 612 sb.append(", seqid=").append(getMaxSequenceId()); 613 } 614 sb.append(", majorCompaction=").append(isMajorCompactionResult()); 615 616 return sb.toString(); 617 } 618 619 /** 620 * Gets whether to skip resetting the sequence id for cells. 621 * @param skipResetSeqId The byte array of boolean. 622 * @return Whether to skip resetting the sequence id. 623 */ 624 private boolean isSkipResetSeqId(byte[] skipResetSeqId) { 625 if (skipResetSeqId != null && skipResetSeqId.length == 1) { 626 return Bytes.toBoolean(skipResetSeqId); 627 } 628 return false; 629 } 630 631 @Override 632 public OptionalLong getMinimumTimestamp() { 633 TimeRange tr = getReader().timeRange; 634 return tr != null ? OptionalLong.of(tr.getMin()) : OptionalLong.empty(); 635 } 636 637 @Override 638 public OptionalLong getMaximumTimestamp() { 639 TimeRange tr = getReader().timeRange; 640 return tr != null ? OptionalLong.of(tr.getMax()) : OptionalLong.empty(); 641 } 642 643 Set<String> getCompactedStoreFiles() { 644 return Collections.unmodifiableSet(this.compactedStoreFiles); 645 } 646 647 long increaseRefCount() { 648 return this.fileInfo.increaseRefCount(); 649 } 650 651 long decreaseRefCount() { 652 return this.fileInfo.decreaseRefCount(); 653 } 654 655 static void increaseStoreFilesRefeCount(Collection<HStoreFile> storeFiles) { 656 if (CollectionUtils.isEmpty(storeFiles)) { 657 return; 658 } 659 storeFiles.forEach(HStoreFile::increaseRefCount); 660 } 661 662 static void decreaseStoreFilesRefeCount(Collection<HStoreFile> storeFiles) { 663 if (CollectionUtils.isEmpty(storeFiles)) { 664 return; 665 } 666 storeFiles.forEach(HStoreFile::decreaseRefCount); 667 } 668}