Class HStoreFile
java.lang.Object
org.apache.hadoop.hbase.regionserver.HStoreFile
- All Implemented Interfaces:
StoreFile
A Store data file. Stores usually have one or more of these files. They are produced by flushing
the memstore to disk. To create, instantiate a writer using
StoreFileWriter.Builder
and
append data. Be sure to add any metadata before calling close on the Writer (Use the
appendMetadata convenience methods). On close, a StoreFile is sitting in the Filesystem. To refer
to it, create a StoreFile instance passing filesystem and path. To read, call
initReader()
StoreFiles may also reference store files in another Store. The reason for this weird pattern where you use a different instance for the writer and a reader is that we write once but read a lot more.
-
Field Summary
Modifier and TypeFieldDescriptionstatic final byte[]
Bloom filter param in FileInfostatic final byte[]
Bloom filter Type in FileInfostatic final byte[]
Meta key set when store file is a result of a bulk loadstatic final byte[]
private final CacheConfig
private final BloomType
Bloom filter type specified in column family configuration.private boolean
static final byte[]
Key for compaction event which contains the compacted storefiles in FileInfoprivate CellComparator
static final byte[]
Delete Family Count in FileInfostatic final byte[]
Key for timestamp of earliest-put in metadatastatic final byte[]
Minor compaction flag in FileInfoprivate boolean
private final StoreFileInfo
private Optional<ExtendedCell>
static final byte[]
private StoreFileReader
private InputStreamBlockDistribution
private boolean
static final byte[]
Last Bloom filter key in FileInfoprivate Optional<ExtendedCell>
private static final org.slf4j.Logger
static final byte[]
Major compaction flag in FileInfoprivate AtomicBoolean
static final byte[]
Max Sequence ID in FileInfoprivate long
private Map<byte[],
byte[]> Map of the metadata entries in the corresponding HFile.private final BloomFilterMetrics
static final byte[]
Key for the number of mob cells in metadatastatic final byte[]
Key for the list of MOB file referencesstatic final byte[]
Null dataprivate long
static final byte[]
Key for skipping resetting sequence id in metadata.static final byte[]
Key for Timerange information in metadata -
Constructor Summary
ConstructorDescriptionHStoreFile
(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, org.apache.hadoop.conf.Configuration conf, CacheConfig cacheConf, BloomType cfBloomType, boolean primaryReplica, StoreFileTracker sft) Constructor, loads a reader and it's indices, etc.HStoreFile
(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf) Constructor, loads a reader and it's indices, etc.HStoreFile
(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf, BloomFilterMetrics metrics) Constructor, loads a reader and it's indices, etc. -
Method Summary
Modifier and TypeMethodDescriptionvoid
closeStoreFile
(boolean evictOnClose) private StoreFileReader
createStreamReader
(boolean canUseDropBehind) (package private) long
(package private) static void
decreaseStoreFilesRefeCount
(Collection<HStoreFile> storeFiles) void
Delete this fileboolean
Returns True if this file should not be part of a minor compaction.Return the timestamp at which this bulk load file was generated.Get the comparator for comparing two cells.org.apache.hadoop.fs.Path
Returns Encoded Path if this StoreFile was made with a Stream.Get the first key in this store file.Get the last key in this store file.Get the max timestamp of all the cells in the store file.long
Get max of the MemstoreTS in the KV's in this store file.long
Returns This files maximum edit sequence id.byte[]
getMetadataValue
(byte[] key) Get the min timestamp of all the cells in the store file.long
Get the modification time of this store file.org.apache.hadoop.fs.Path
getPath()
Returns Path or null if this StoreFile was made with a Stream.getPreadScanner
(boolean cacheBlocks, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) Get a scanner which uses pread.org.apache.hadoop.fs.Path
Returns Returns the qualified path of this StoreFileint
getStreamScanner
(boolean canUseDropBehind, boolean cacheBlocks, boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) Get a scanner which uses streaming read.(package private) long
(package private) static void
increaseStoreFilesRefeCount
(Collection<HStoreFile> storeFiles) void
Initialize the reader used for pread.boolean
Check if this storefile was created by bulk load.boolean
boolean
isHFile()
Returns True if this is HFile.boolean
boolean
Returns True if this file was made by a major compaction.boolean
Returns True if this is a StoreFile Reference.boolean
Returns true if the file is still used in readsprivate boolean
isSkipResetSeqId
(byte[] skipResetSeqId) Gets whether to skip resetting the sequence id for cells.void
private void
open()
Opens reader on this store file.toString()
Returns a length description of this StoreFile, suitable for debug output
-
Field Details
-
LOG
-
MAX_SEQ_ID_KEY
Max Sequence ID in FileInfo -
MAJOR_COMPACTION_KEY
Major compaction flag in FileInfo -
EXCLUDE_FROM_MINOR_COMPACTION_KEY
Minor compaction flag in FileInfo -
COMPACTION_EVENT_KEY
Key for compaction event which contains the compacted storefiles in FileInfo -
BLOOM_FILTER_TYPE_KEY
Bloom filter Type in FileInfo -
BLOOM_FILTER_PARAM_KEY
Bloom filter param in FileInfo -
DELETE_FAMILY_COUNT
Delete Family Count in FileInfo -
LAST_BLOOM_KEY
Last Bloom filter key in FileInfo -
TIMERANGE_KEY
Key for Timerange information in metadata -
EARLIEST_PUT_TS
Key for timestamp of earliest-put in metadata -
MOB_CELLS_COUNT
Key for the number of mob cells in metadata -
NULL_VALUE
Null data -
MOB_FILE_REFS
Key for the list of MOB file references -
BULKLOAD_TASK_KEY
Meta key set when store file is a result of a bulk load -
BULKLOAD_TIME_KEY
-
SKIP_RESET_SEQ_ID
Key for skipping resetting sequence id in metadata. For bulk loaded hfiles, the scanner resets the cell seqId with the latest one, if this metadata is set as true, the reset is skipped. -
HISTORICAL_KEY
-
fileInfo
-
initialReader
-
initialReaderBlockDistribution
-
cacheConf
-
metrics
-
compactedAway
-
isHistorical
-
sequenceid
-
maxMemstoreTS
-
firstKey
-
lastKey
-
comparator
-
majorCompaction
-
excludeFromMinorCompaction
-
compactedStoreFiles
-
metadataMap
Map of the metadata entries in the corresponding HFile. Populated when Reader is opened after which it is not modified again. -
cfBloomType
Bloom filter type specified in column family configuration. Does not necessarily correspond to the Bloom filter type present in the HFile.
-
-
Constructor Details
-
HStoreFile
public HStoreFile(org.apache.hadoop.fs.FileSystem fs, org.apache.hadoop.fs.Path p, org.apache.hadoop.conf.Configuration conf, CacheConfig cacheConf, BloomType cfBloomType, boolean primaryReplica, StoreFileTracker sft) throws IOException Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram depending on the underlying files (10-20MB?).- Parameters:
fs
- The current file system to use.p
- The path of the file.conf
- The current configuration.cacheConf
- The cache configuration and block cache reference.cfBloomType
- The bloom type to use for this store file as specified by column family configuration. This may or may not be the same as the Bloom filter type actually present in the HFile, because column family configuration might change. If this isBloomType.NONE
, the existing Bloom filter is ignored.primaryReplica
- true if this is a store file for primary replica, otherwise false.- Throws:
IOException
-
HStoreFile
Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram depending on the underlying files (10-20MB?).- Parameters:
fileInfo
- The store file information.cfBloomType
- The bloom type to use for this store file as specified by column family configuration. This may or may not be the same as the Bloom filter type actually present in the HFile, because column family configuration might change. If this isBloomType.NONE
, the existing Bloom filter is ignored.cacheConf
- The cache configuration and block cache reference.
-
HStoreFile
public HStoreFile(StoreFileInfo fileInfo, BloomType cfBloomType, CacheConfig cacheConf, BloomFilterMetrics metrics) Constructor, loads a reader and it's indices, etc. May allocate a substantial amount of ram depending on the underlying files (10-20MB?).- Parameters:
fileInfo
- The store file information.cfBloomType
- The bloom type to use for this store file as specified by column family configuration. This may or may not be the same as the Bloom filter type actually present in the HFile, because column family configuration might change. If this isBloomType.NONE
, the existing Bloom filter is ignored.cacheConf
- The cache configuration and block cache reference.metrics
- Tracks bloom filter requests and results. May be null.
-
-
Method Details
-
getCacheConf
-
getFirstKey
Description copied from interface:StoreFile
Get the first key in this store file.- Specified by:
getFirstKey
in interfaceStoreFile
-
getLastKey
Description copied from interface:StoreFile
Get the last key in this store file.- Specified by:
getLastKey
in interfaceStoreFile
-
getComparator
Description copied from interface:StoreFile
Get the comparator for comparing two cells.- Specified by:
getComparator
in interfaceStoreFile
-
getMaxMemStoreTS
Description copied from interface:StoreFile
Get max of the MemstoreTS in the KV's in this store file.- Specified by:
getMaxMemStoreTS
in interfaceStoreFile
-
getFileInfo
- Returns:
- the StoreFile object associated to this StoreFile. null if the StoreFile is not a reference.
-
getPath
Description copied from interface:StoreFile
Returns Path or null if this StoreFile was made with a Stream. -
getEncodedPath
Description copied from interface:StoreFile
Returns Encoded Path if this StoreFile was made with a Stream.- Specified by:
getEncodedPath
in interfaceStoreFile
-
getQualifiedPath
Description copied from interface:StoreFile
Returns Returns the qualified path of this StoreFile- Specified by:
getQualifiedPath
in interfaceStoreFile
-
isReference
Description copied from interface:StoreFile
Returns True if this is a StoreFile Reference.- Specified by:
isReference
in interfaceStoreFile
-
isHFile
Description copied from interface:StoreFile
Returns True if this is HFile. -
isMajorCompactionResult
Description copied from interface:StoreFile
Returns True if this file was made by a major compaction.- Specified by:
isMajorCompactionResult
in interfaceStoreFile
-
excludeFromMinorCompaction
Description copied from interface:StoreFile
Returns True if this file should not be part of a minor compaction.- Specified by:
excludeFromMinorCompaction
in interfaceStoreFile
-
getMaxSequenceId
Description copied from interface:StoreFile
Returns This files maximum edit sequence id.- Specified by:
getMaxSequenceId
in interfaceStoreFile
-
getModificationTimestamp
Description copied from interface:StoreFile
Get the modification time of this store file. Usually will access the file system so throws IOException.- Specified by:
getModificationTimestamp
in interfaceStoreFile
- Throws:
IOException
-
getMetadataValue
- Parameters:
key
- to look up- Returns:
- value associated with the metadata key
-
isBulkLoadResult
Description copied from interface:StoreFile
Check if this storefile was created by bulk load. When a hfile is bulk loaded into HBase, we append'_SeqId_<id-when-loaded>'
to the hfile name, unless "hbase.mapreduce.bulkload.assign.sequenceNumbers" is explicitly turned off. If "hbase.mapreduce.bulkload.assign.sequenceNumbers" is turned off, fall back to BULKLOAD_TIME_KEY.- Specified by:
isBulkLoadResult
in interfaceStoreFile
- Returns:
- true if this storefile was created by bulk load.
-
isCompactedAway
-
isHistorical
-
getRefCount
-
isReferencedInReads
Returns true if the file is still used in reads -
getBulkLoadTimestamp
Description copied from interface:StoreFile
Return the timestamp at which this bulk load file was generated.- Specified by:
getBulkLoadTimestamp
in interfaceStoreFile
-
getHDFSBlockDistribution
- Returns:
- the cached value of HDFS blocks distribution. The cached value is calculated when store file is opened.
-
open
Opens reader on this store file. Called by Constructor.- Throws:
IOException
- See Also:
-
initReader
Initialize the reader used for pread.- Throws:
IOException
-
createStreamReader
- Throws:
IOException
-
getPreadScanner
public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) Get a scanner which uses pread.Must be called after initReader.
-
getStreamScanner
public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks, boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn) throws IOException Get a scanner which uses streaming read.Must be called after initReader.
- Throws:
IOException
-
getReader
- Returns:
- Current reader. Must call initReader first else returns null.
- See Also:
-
closeStoreFile
- Parameters:
evictOnClose
- whether to evict blocks belonging to this file- Throws:
IOException
-
deleteStoreFile
Delete this file- Throws:
IOException
-
markCompactedAway
-
toString
-
toStringDetailed
Description copied from interface:StoreFile
Returns a length description of this StoreFile, suitable for debug output- Specified by:
toStringDetailed
in interfaceStoreFile
-
isSkipResetSeqId
Gets whether to skip resetting the sequence id for cells.- Parameters:
skipResetSeqId
- The byte array of boolean.- Returns:
- Whether to skip resetting the sequence id.
-
getMinimumTimestamp
Description copied from interface:StoreFile
Get the min timestamp of all the cells in the store file.- Specified by:
getMinimumTimestamp
in interfaceStoreFile
-
getMaximumTimestamp
Description copied from interface:StoreFile
Get the max timestamp of all the cells in the store file.- Specified by:
getMaximumTimestamp
in interfaceStoreFile
-
getCompactedStoreFiles
-
increaseRefCount
long increaseRefCount() -
decreaseRefCount
long decreaseRefCount() -
increaseStoreFilesRefeCount
-
decreaseStoreFilesRefeCount
-