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}