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