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.hbase.thirdparty.com.google.common.base.Preconditions;
052
053import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
054
055/**
056 * A Store data file. Stores usually have one or more of these files. They are produced by flushing
057 * the memstore to disk. To create, instantiate a writer using {@link StoreFileWriter.Builder} and
058 * append data. Be sure to add any metadata before calling close on the Writer (Use the
059 * appendMetadata convenience methods). On close, a StoreFile is sitting in the Filesystem. To refer
060 * to it, create a StoreFile instance passing filesystem and path. To read, call
061 * {@link #initReader()}
062 * <p>
063 * StoreFiles may also reference store files in another Store. The reason for this weird pattern
064 * where you use a different instance for the writer and a reader is that we write once but read a
065 * lot more.
066 */
067@InterfaceAudience.Private
068public class HStoreFile implements StoreFile {
069
070  private static final Logger LOG = LoggerFactory.getLogger(HStoreFile.class.getName());
071
072  // Keys for fileinfo values in HFile
073
074  /** Max Sequence ID in FileInfo */
075  public static final byte[] MAX_SEQ_ID_KEY = Bytes.toBytes("MAX_SEQ_ID_KEY");
076
077  /** Major compaction flag in FileInfo */
078  public static final byte[] MAJOR_COMPACTION_KEY = Bytes.toBytes("MAJOR_COMPACTION_KEY");
079
080  /** Minor compaction flag in FileInfo */
081  public static final byte[] EXCLUDE_FROM_MINOR_COMPACTION_KEY =
082    Bytes.toBytes("EXCLUDE_FROM_MINOR_COMPACTION");
083
084  /**
085   * Key for compaction event which contains the compacted storefiles in FileInfo
086   */
087  public static final byte[] COMPACTION_EVENT_KEY = Bytes.toBytes("COMPACTION_EVENT_KEY");
088
089  /** Bloom filter Type in FileInfo */
090  public static final byte[] BLOOM_FILTER_TYPE_KEY = Bytes.toBytes("BLOOM_FILTER_TYPE");
091
092  /** Bloom filter param in FileInfo */
093  public static final byte[] BLOOM_FILTER_PARAM_KEY = Bytes.toBytes("BLOOM_FILTER_PARAM");
094
095  /** Delete Family Count in FileInfo */
096  public static final byte[] DELETE_FAMILY_COUNT = Bytes.toBytes("DELETE_FAMILY_COUNT");
097
098  /** Last Bloom filter key in FileInfo */
099  public static final byte[] LAST_BLOOM_KEY = Bytes.toBytes("LAST_BLOOM_KEY");
100
101  /** Key for Timerange information in metadata */
102  public static final byte[] TIMERANGE_KEY = Bytes.toBytes("TIMERANGE");
103
104  /** Key for timestamp of earliest-put in metadata */
105  public static final byte[] EARLIEST_PUT_TS = Bytes.toBytes("EARLIEST_PUT_TS");
106
107  /** Key for the number of mob cells in metadata */
108  public static final byte[] MOB_CELLS_COUNT = Bytes.toBytes("MOB_CELLS_COUNT");
109
110  /** Null data */
111  public static final byte[] NULL_VALUE = new byte[] { 0 };
112
113  /** Key for the list of MOB file references */
114  public static final byte[] MOB_FILE_REFS = Bytes.toBytes("MOB_FILE_REFS");
115
116  /** Meta key set when store file is a result of a bulk load */
117  public static final byte[] BULKLOAD_TASK_KEY = Bytes.toBytes("BULKLOAD_SOURCE_TASK");
118  public static final byte[] BULKLOAD_TIME_KEY = Bytes.toBytes("BULKLOAD_TIMESTAMP");
119
120  /**
121   * Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets
122   * the cell seqId with the latest one, if this metadata is set as true, the reset is skipped.
123   */
124  public static final byte[] SKIP_RESET_SEQ_ID = Bytes.toBytes("SKIP_RESET_SEQ_ID");
125
126  private final StoreFileInfo fileInfo;
127
128  // StoreFile.Reader
129  private volatile StoreFileReader initialReader;
130  private volatile InputStreamBlockDistribution initialReaderBlockDistribution = null;
131
132  // Block cache configuration and reference.
133  private final CacheConfig cacheConf;
134  private final BloomFilterMetrics metrics;
135
136  // Indicates if the file got compacted
137  private volatile boolean compactedAway = false;
138
139  // Keys for metadata stored in backing HFile.
140  // Set when we obtain a Reader.
141  private long sequenceid = -1;
142
143  // max of the MemstoreTS in the KV's in this store
144  // Set when we obtain a Reader.
145  private long maxMemstoreTS = -1;
146
147  // firstKey, lastkey and cellComparator will be set when openReader.
148  private Optional<Cell> firstKey;
149
150  private Optional<Cell> lastKey;
151
152  private CellComparator comparator;
153
154  public CacheConfig getCacheConf() {
155    return this.cacheConf;
156  }
157
158  @Override
159  public Optional<Cell> getFirstKey() {
160    return firstKey;
161  }
162
163  @Override
164  public Optional<Cell> getLastKey() {
165    return lastKey;
166  }
167
168  @Override
169  public CellComparator getComparator() {
170    return comparator;
171  }
172
173  @Override
174  public long getMaxMemStoreTS() {
175    return maxMemstoreTS;
176  }
177
178  // If true, this file was product of a major compaction. Its then set
179  // whenever you get a Reader.
180  private AtomicBoolean majorCompaction = null;
181
182  // If true, this file should not be included in minor compactions.
183  // It's set whenever you get a Reader.
184  private boolean excludeFromMinorCompaction = false;
185
186  // This file was product of these compacted store files
187  private final Set<String> compactedStoreFiles = new HashSet<>();
188
189  /**
190   * Map of the metadata entries in the corresponding HFile. Populated when Reader is opened after
191   * which it is not modified again.
192   */
193  private Map<byte[], byte[]> metadataMap;
194
195  /**
196   * Bloom filter type specified in column family configuration. Does not necessarily correspond to
197   * the Bloom filter type present in the HFile.
198   */
199  private final BloomType cfBloomType;
200
201  /**
202   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
203   * depending on the underlying files (10-20MB?).
204   * @param fs             The current file system to use.
205   * @param p              The path of the file.
206   * @param conf           The current configuration.
207   * @param cacheConf      The cache configuration and block cache reference.
208   * @param cfBloomType    The bloom type to use for this store file as specified by column family
209   *                       configuration. This may or may not be the same as the Bloom filter type
210   *                       actually present in the HFile, because column family configuration might
211   *                       change. If this is {@link BloomType#NONE}, the existing Bloom filter is
212   *                       ignored.
213   * @param primaryReplica true if this is a store file for primary replica, otherwise false.
214   */
215  public HStoreFile(FileSystem fs, Path p, Configuration conf, CacheConfig cacheConf,
216    BloomType cfBloomType, boolean primaryReplica) throws IOException {
217    this(new StoreFileInfo(conf, fs, p, primaryReplica), cfBloomType, cacheConf);
218  }
219
220  /**
221   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
222   * depending on the underlying files (10-20MB?).
223   * @param fileInfo    The store file information.
224   * @param cfBloomType The bloom type to use for this store file as specified by column family
225   *                    configuration. This may or may not be the same as the Bloom filter type
226   *                    actually present in the HFile, because column family configuration might
227   *                    change. If this is {@link BloomType#NONE}, the existing Bloom filter is
228   *                    ignored.
229   * @param cacheConf   The cache configuration and block cache reference.
230   */
231  public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf) {
232    this(fileInfo, cfBloomType, cacheConf, null);
233  }
234
235  /**
236   * Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram
237   * depending on the underlying files (10-20MB?).
238   * @param fileInfo    The store file information.
239   * @param cfBloomType The bloom type to use for this store file as specified by column family
240   *                    configuration. This may or may not be the same as the Bloom filter type
241   *                    actually present in the HFile, because column family configuration might
242   *                    change. If this is {@link BloomType#NONE}, the existing Bloom filter is
243   *                    ignored.
244   * @param cacheConf   The cache configuration and block cache reference.
245   * @param metrics     Tracks bloom filter requests and results. May be null.
246   */
247  public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf,
248    BloomFilterMetrics metrics) {
249    this.fileInfo = fileInfo;
250    this.cacheConf = cacheConf;
251    this.metrics = metrics;
252    if (BloomFilterFactory.isGeneralBloomEnabled(fileInfo.getConf())) {
253      this.cfBloomType = cfBloomType;
254    } else {
255      LOG.info("Ignoring bloom filter check for file " + this.getPath() + ": " + "cfBloomType="
256        + cfBloomType + " (disabled in config)");
257      this.cfBloomType = BloomType.NONE;
258    }
259  }
260
261  /**
262   * @return the StoreFile object associated to this StoreFile. null if the StoreFile is not a
263   *         reference.
264   */
265  public StoreFileInfo getFileInfo() {
266    return this.fileInfo;
267  }
268
269  @Override
270  public Path getPath() {
271    return this.fileInfo.getPath();
272  }
273
274  @Override
275  public Path getEncodedPath() {
276    try {
277      return new Path(URLEncoder.encode(fileInfo.getPath().toString(), HConstants.UTF8_ENCODING));
278    } catch (UnsupportedEncodingException ex) {
279      throw new RuntimeException("URLEncoder doesn't support UTF-8", ex);
280    }
281  }
282
283  @Override
284  public Path getQualifiedPath() {
285    FileSystem fs = fileInfo.getFileSystem();
286    return this.fileInfo.getPath().makeQualified(fs.getUri(), fs.getWorkingDirectory());
287  }
288
289  @Override
290  public boolean isReference() {
291    return this.fileInfo.isReference();
292  }
293
294  @Override
295  public boolean isHFile() {
296    return StoreFileInfo.isHFile(this.fileInfo.getPath());
297  }
298
299  @Override
300  public boolean isMajorCompactionResult() {
301    Preconditions.checkState(this.majorCompaction != null, "Major compation has not been set yet");
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 fileInfo.getModificationTime();
318  }
319
320  /**
321   * @param key to look up
322   * @return value associated with the metadata key
323   */
324  public byte[] getMetadataValue(byte[] key) {
325    return metadataMap.get(key);
326  }
327
328  @Override
329  public boolean isBulkLoadResult() {
330    boolean bulkLoadedHFile = false;
331    String fileName = this.getPath().getName();
332    int startPos = fileName.indexOf("SeqId_");
333    if (startPos != -1) {
334      bulkLoadedHFile = true;
335    }
336    return bulkLoadedHFile || (metadataMap != null && metadataMap.containsKey(BULKLOAD_TIME_KEY));
337  }
338
339  public boolean isCompactedAway() {
340    return compactedAway;
341  }
342
343  public int getRefCount() {
344    return fileInfo.refCount.get();
345  }
346
347  /** Returns true if the file is still used in reads */
348  public boolean isReferencedInReads() {
349    int rc = fileInfo.refCount.get();
350    assert rc >= 0; // we should not go negative.
351    return rc > 0;
352  }
353
354  @Override
355  public OptionalLong getBulkLoadTimestamp() {
356    byte[] bulkLoadTimestamp = metadataMap.get(BULKLOAD_TIME_KEY);
357    return bulkLoadTimestamp == null
358      ? OptionalLong.empty()
359      : OptionalLong.of(Bytes.toLong(bulkLoadTimestamp));
360  }
361
362  /**
363   * @return the cached value of HDFS blocks distribution. The cached value is calculated when store
364   *         file is opened.
365   */
366  public HDFSBlocksDistribution getHDFSBlockDistribution() {
367    if (initialReaderBlockDistribution != null) {
368      return initialReaderBlockDistribution.getHDFSBlockDistribution();
369    } else {
370      return this.fileInfo.getHDFSBlockDistribution();
371    }
372  }
373
374  /**
375   * Opens reader on this store file. Called by Constructor.
376   * @see #closeStoreFile(boolean)
377   */
378  private void open() throws IOException {
379    fileInfo.initHDFSBlocksDistribution();
380    long readahead = fileInfo.isNoReadahead() ? 0L : -1L;
381    ReaderContext context = fileInfo.createReaderContext(false, readahead, ReaderType.PREAD);
382    fileInfo.initHFileInfo(context);
383    StoreFileReader reader = fileInfo.preStoreFileReaderOpen(context, cacheConf);
384    if (reader == null) {
385      reader = fileInfo.createReader(context, cacheConf);
386      fileInfo.getHFileInfo().initMetaAndIndex(reader.getHFileReader());
387    }
388    this.initialReader = fileInfo.postStoreFileReaderOpen(context, cacheConf, reader);
389
390    if (InputStreamBlockDistribution.isEnabled(fileInfo.getConf())) {
391      boolean useHBaseChecksum = context.getInputStreamWrapper().shouldUseHBaseChecksum();
392      FSDataInputStream stream = context.getInputStreamWrapper().getStream(useHBaseChecksum);
393      this.initialReaderBlockDistribution = new InputStreamBlockDistribution(stream, fileInfo);
394    }
395
396    // Load up indices and fileinfo. This also loads Bloom filter type.
397    metadataMap = Collections.unmodifiableMap(initialReader.loadFileInfo());
398
399    // Read in our metadata.
400    byte[] b = metadataMap.get(MAX_SEQ_ID_KEY);
401    if (b != null) {
402      // By convention, if halfhfile, top half has a sequence number > bottom
403      // half. Thats why we add one in below. Its done for case the two halves
404      // are ever merged back together --rare. Without it, on open of store,
405      // since store files are distinguished by sequence id, the one half would
406      // subsume the other.
407      this.sequenceid = Bytes.toLong(b);
408      if (fileInfo.isTopReference()) {
409        this.sequenceid += 1;
410      }
411    }
412
413    if (isBulkLoadResult()) {
414      // generate the sequenceId from the fileName
415      // fileName is of the form <randomName>_SeqId_<id-when-loaded>_
416      String fileName = this.getPath().getName();
417      // Use lastIndexOf() to get the last, most recent bulk load seqId.
418      int startPos = fileName.lastIndexOf("SeqId_");
419      if (startPos != -1) {
420        this.sequenceid =
421          Long.parseLong(fileName.substring(startPos + 6, fileName.indexOf('_', startPos + 6)));
422        // Handle reference files as done above.
423        if (fileInfo.isTopReference()) {
424          this.sequenceid += 1;
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}