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