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