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