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