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