Class HStore
- All Implemented Interfaces:
ConfigurationObserver
,PropagatingConfigurationObserver
,HeapSize
,Store
,StoreConfigInformation
- Direct Known Subclasses:
HMobStore
There's no reason to consider append-logging at this level; all logging and locking is handled at the HRegion level. Store just provides services to manage sets of StoreFiles. One of the most important of those services is compaction services where files are aggregated once they pass a configurable threshold.
Locking and transactions are handled at a higher level. This API should not be called directly but by an HRegion manager.
-
Nested Class Summary
Modifier and TypeClassDescriptionprivate static final class
private final class
-
Field Summary
Modifier and TypeFieldDescription(package private) final ReentrantLock
Lock specific to archiving compacted store files.static final String
static final String
private long
private final Set<ChangedReadersObserver>
static final String
private AtomicLong
private AtomicLong
private int
protected org.apache.hadoop.conf.Configuration
private final AtomicInteger
Use this counter to track concurrent puts.private HFileDataBlockEncoder
static final long
static final String
static final int
static final int
private static final int
private final List<HStoreFile>
static final long
private AtomicLong
private AtomicLong
private AtomicLong
private int
(package private) boolean
private long
private static final org.slf4j.Logger
private AtomicLong
private AtomicLong
protected final MemStore
static final String
private LongAdder
private LongAdder
private static final AtomicBoolean
private OffPeakHours
private final int
private int
private final HRegion
private ScanInfo
private static final int
private final StoreContext
(package private) final StoreEngine<?,
?, ?, ?> private final Supplier<HStore.StoreFileWriterCreationTracker>
private final Set<HStore.StoreFileWriterCreationTracker>
private AtomicLong
private AtomicLong
private final boolean
private final boolean
Fields inherited from interface org.apache.hadoop.hbase.regionserver.Store
NO_PRIORITY, PRIORITY_USER
-
Constructor Summary
ModifierConstructorDescriptionprotected
HStore
(HRegion region, ColumnFamilyDescriptor family, org.apache.hadoop.conf.Configuration confParam, boolean warmup) Constructor -
Method Summary
Modifier and TypeMethodDescriptionvoid
add
(Iterable<ExtendedCell> cells, MemStoreSizing memstoreSizing) Adds the specified value to the memstorevoid
add
(ExtendedCell cell, MemStoreSizing memstoreSizing) Adds a value to the memstorevoid
private void
addToCompactingFiles
(Collection<HStoreFile> filesToAdd) Adds the files to compacting files.boolean
void
assertBulkLoadHFileOk
(org.apache.hadoop.fs.Path srcPath) This throws a WrongRegionException if the HFile does not fit in this region, or an InvalidHFileException if the HFile is not valid.org.apache.hadoop.fs.Path
bulkLoadHFile
(byte[] family, String srcPathStr, org.apache.hadoop.fs.Path dstPath) private void
void
bulkLoadHFile
(StoreFileInfo fileInfo) void
cancelRequestedCompaction
(CompactionContext compaction) boolean
canSplit()
Returns whether this store is splittable, i.e., no reference file in this store.private static void
clearAndClose
(List<KeyValueScanner> scanners) private void
clearCompactedfiles
(List<HStoreFile> filesToRemove) org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection<HStoreFile>
close()
Close all the readers We don't need to worry about subsequent requests because the Region holds a write lock that will prevent any more reads or writes.void
Closes and archives the compacted files under this storeprivate org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection<HStoreFile>
compact
(CompactionContext compaction, ThroughputController throughputController, User user) Compact the StoreFiles.private boolean
completeFlush
(List<HStoreFile> sfs, long snapshotId) protected CacheConfig
Creates the cache config.createFlushContext
(long cacheFlushId, FlushLifeCycleTracker tracker) protected KeyValueScanner
createScanner
(Scan scan, ScanInfo scanInfo, NavigableSet<byte[]> targetCols, long readPt) protected StoreEngine<?,
?, ?, ?> createStoreEngine
(HStore store, org.apache.hadoop.conf.Configuration conf, CellComparator kvComparator) Creates the store engine configured for the given Store.void
void
deregisterChildren
(ConfigurationManager manager) Needs to be called to deregister the children from the manager.static long
Returns TTL in seconds of the specified familyprotected List<HStoreFile>
doCompaction
(CompactionRequestImpl cr, Collection<HStoreFile> filesToCompact, User user, long compactionStartTime, List<org.apache.hadoop.fs.Path> newFiles) private void
protected List<org.apache.hadoop.fs.Path>
flushCache
(long logCacheFlushId, MemStoreSnapshot snapshot, MonitoredTask status, ThroughputController throughputController, FlushLifeCycleTracker tracker, Consumer<org.apache.hadoop.fs.Path> writerCreationTracker) Write out current snapshot.Returns Average age of store files in this storelong
The number of files required before flushes for this store will be blocked.long
Returns count of requests which could have used bloom filters, but they weren't configured or loaded.long
Returns count of negative results for bloom filter requests for this store.long
Returns count of bloom filter results for this store.Used for tests.long
Returns The number of cells processed during minor compactionslong
Returns The total amount of data processed during minor compactions, in bytesint
Returns Count of compacted store fileslong
double
This value can represent the degree of emergency of compaction for this store.getter for CompactionProgress objectint
int
Returns the data block encoderprivate InetSocketAddress[]
org.apache.hadoop.fs.FileSystem
long
Returns The number of cells flushed to disklong
Returns The total size of data flushed to disk, in byteslong
Returns The total size of out output files on disk, in byteslong
Returns The size of only the store files which are HFiles, in bytes.long
Returns aggregate size of all HStores used in the last compactionlong
Returns The number of cells processed during major compactionslong
Returns The total amount of data processed during major compactions, in bytesint
Returns get maximum ref count of storeFile among all compacted HStore Files for the HStoreReturns The maximum memstoreTS in all store files.Returns The maximum sequence id in all store files.Returns Max age of store files in this storeprivate MemStore
Returns MemStore Instance to use in this store.long
Returns Gets the Memstore flush size for the region that this store works with.long
Returns the number of read requests purely from the memstore.Returns The size of this store's memstore.Returns Min age of store files in this storelong
Returns the number of read requests from the files under this store.long
Returns Number of HFiles in this storelong
Returns Number of reference files in this storeprotected OffPeakHours
org.apache.hadoop.conf.Configuration
Returns the parent region info hosting this storegetScanner
(Scan scan, NavigableSet<byte[]> targetCols, long readPt) Return a scanner for both the memstore and the HStore files.getScanners
(boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt, boolean onlyLatestVersion) Get all scanners with no filtering based on TTL (that happens further down the line).getScanners
(boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean onlyLatestVersion) Get all scanners with no filtering based on TTL (that happens further down the line).getScanners
(List<HStoreFile> files, boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner, boolean onlyLatestVersion) Create scanners on the given files and if needed on the memstore with no filtering based on TTL (that happens further down the line).getScanners
(List<HStoreFile> files, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner, boolean onlyLatestVersion) Create scanners on the given files and if needed on the memstore with no filtering based on TTL (that happens further down the line).long
getSize()
Returns aggregate size of HStorelong
Returns size of the memstore snapshotOptional<byte[]>
Determines if Store should be split.StoreEngine<?,
?, ?, ?> Returns the StoreEngine that is backing this concrete implementation of Store.private LongStream
Returns All store files.Set<org.apache.hadoop.fs.Path>
Return the storefiles which are currently being written to.int
Returns Count of store filesprivate long
(package private) long
getStoreFileSize
(HStoreFile file) Computes the length of a store file without succumbing to any errors along the way.long
Returns The size of the store file root-level indexes, in bytes.long
Returns The size of the store files, in bytes.long
Returns Gets the cf-specific time-to-live for store files.int
long
Returns The size of the store files, in bytes, uncompressed.private long
getTotalSize
(Collection<HStoreFile> sfs) long
Returns the total byte size of all Bloom filter bit arrays.long
Returns the total size of all index blocks in the data block indexes, including the root level, intermediate levels, and the leaf level for multi-level indexes, or just the root level for single-level indexes.boolean
Returnstrue
if the store has any underlying reference files to older HFilesboolean
Returns Whether this store has too many store files.long
heapSize()
Return the approximate 'exclusive deep size' of implementing object.private StoreContext
boolean
boolean
Returns true if the memstore may need some extra memory spaceprivate void
logCompactionEndMessage
(CompactionRequestImpl cr, List<HStoreFile> sfs, long now, long compactionStartTime) Log a very elaborate compaction completion message.boolean
See if there's too much store files in this storeprivate void
Notify all observers that set of Readers has changed.void
onConfigurationChange
(org.apache.hadoop.conf.Configuration conf) This method would be called by theConfigurationManager
object when theConfiguration
object is reloaded from disk.private void
void
Perform tasks needed after the completion of snapshot operation.Pair<org.apache.hadoop.fs.Path,
org.apache.hadoop.fs.Path> preBulkLoadHFile
(String srcPathStr, long seqNum) This method should only be called from Region.void
Sets the store up for a region level snapshot operation.recreateScanners
(List<KeyValueScanner> currentFileScanners, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) Recreates the scanners on the current list of active store file scannersvoid
Checks the underlying store files, and opens the files that have not been opened, and removes the store file readers for store files no longer available.void
refreshStoreFiles
(Collection<String> newFiles) Replaces the store files that the store has with the given files.protected void
Update counts.void
registerChildren
(ConfigurationManager manager) Needs to be called to register the children to the manager.private void
removeCompactedfiles
(Collection<HStoreFile> compactedfiles, boolean evictOnClose) Archives and removes the compacted filesprivate void
(package private) void
replaceStoreFiles
(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result, boolean writeCompactionMarker) void
replayCompactionMarker
(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor compaction, boolean pickCompactionFiles, boolean removeFiles) Call to complete a compaction.(package private) void
reportArchivedFilesForQuota
(List<? extends StoreFile> archivedFiles, List<Long> fileSizes) requestCompaction
(int priority, CompactionLifeCycleTracker tracker, User user) (package private) void
setDataBlockEncoderInTest
(HFileDataBlockEncoder blockEncoder) Should be used only in tests.(package private) void
setScanInfo
(ScanInfo scanInfo) Set scan info, used by testprivate void
setStoragePolicyFromFileName
(List<org.apache.hadoop.fs.Path> newFiles) boolean
Tests whether we should run a major compaction.void
This message intends to inform the MemStore that next coming updates are going to be part of the replaying edits from WALvoid
This message intends to inform the MemStore that the replaying edits from WAL are doneboolean
throttleCompaction
(long compactionSize) long
When was the last edit done in the memstoretoString()
void
tryCommitRecoveredHFile
(org.apache.hadoop.fs.Path path) void
updateCompactedMetrics
(boolean isMajor, CompactionProgress progress) (package private) void
updateMetricsStore
(boolean memstoreRead) (package private) void
updateSpaceQuotaAfterFileReplacement
(RegionSizeStore sizeStore, RegionInfo regionInfo, Collection<HStoreFile> oldFiles, Collection<HStoreFile> newFiles) Updates the space quota usage for this region, removing the size for files compacted away and adding in the size for new files.void
upsert
(Iterable<ExtendedCell> cells, long readpoint, MemStoreSizing memstoreSizing) Adds or replaces the specified KeyValues.(package private) int
versionsToReturn
(int wantedVersions) private void
writeCompactionWalRecord
(Collection<HStoreFile> filesCompacted, Collection<HStoreFile> newFiles) Writes the compaction WAL record.
-
Field Details
-
MEMSTORE_CLASS_NAME
- See Also:
-
COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY
- See Also:
-
BLOCKING_STOREFILES_KEY
- See Also:
-
BLOCK_STORAGE_POLICY_KEY
- See Also:
-
DEFAULT_BLOCK_STORAGE_POLICY
- See Also:
-
DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER
- See Also:
-
DEFAULT_BLOCKING_STOREFILE_COUNT
- See Also:
-
SPLIT_REGION_COMPACTION_PRIORITY
- See Also:
-
LOG
-
memstore
-
region
-
conf
-
lastCompactSize
-
forceMajor
-
storeSize
-
totalUncompressedBytes
-
memstoreOnlyRowReadsCount
-
mixedRowReadsCount
-
archiveLock
Lock specific to archiving compacted store files. This avoids races around the combination of retrieving the list of compacted files and moving them to the archive directory. Since this is usually a background process (other than on close), we don't want to handle this with the store write lock, which would block readers and degrade performance. Locked by: - CompactedHFilesDispatchHandler via closeAndArchiveCompactedFiles() - close() -
verifyBulkLoads
-
currentParallelPutCount
Use this counter to track concurrent puts. If TRACE-log is enabled, if we are over the threshold set by hbase.region.store.parallel.put.print.threshold (Default is 50) we will log a message that identifies the Store experience this high-level of concurrency. -
parallelPutCountPrintThreshold
-
scanInfo
-
filesCompacting
-
changedReaderObservers
-
dataBlockEncoder
-
storeEngine
-
offPeakCompactionTracker
-
offPeakHours
-
DEFAULT_FLUSH_RETRIES_NUMBER
- See Also:
-
flushRetriesNumber
-
pauseTime
-
blockingFileCount
-
compactionCheckMultiplier
-
flushedCellsCount
-
compactedCellsCount
-
majorCompactedCellsCount
-
flushedCellsSize
-
flushedOutputFileSize
-
compactedCellsSize
-
majorCompactedCellsSize
-
storeContext
-
storeFileWriterCreationTrackers
-
storeFileWriterCreationTrackerFactory
-
warmup
-
FIXED_OVERHEAD
-
DEEP_OVERHEAD
-
-
Constructor Details
-
HStore
protected HStore(HRegion region, ColumnFamilyDescriptor family, org.apache.hadoop.conf.Configuration confParam, boolean warmup) throws IOException Constructor- Parameters:
family
- HColumnDescriptor for this columnconfParam
- configuration object failed. Can be null.- Throws:
IOException
-
-
Method Details
-
initializeStoreContext
- Throws:
IOException
-
getFavoredNodes
-
getMemstore
Returns MemStore Instance to use in this store. -
createCacheConf
Creates the cache config.- Parameters:
family
- The current column family.
-
createStoreEngine
protected StoreEngine<?,?, createStoreEngine?, ?> (HStore store, org.apache.hadoop.conf.Configuration conf, CellComparator kvComparator) throws IOException Creates the store engine configured for the given Store.- Parameters:
store
- The store. An unfortunate dependency needed due to it being passed to coprocessors via the compactor.conf
- Store configuration.kvComparator
- KVComparator for storeFileManager.- Returns:
- StoreEngine to use.
- Throws:
IOException
-
determineTTLFromFamily
Returns TTL in seconds of the specified family -
getStoreContext
-
getColumnFamilyName
- Specified by:
getColumnFamilyName
in interfaceStore
- Specified by:
getColumnFamilyName
in interfaceStoreConfigInformation
-
getTableName
- Specified by:
getTableName
in interfaceStore
-
getFileSystem
- Specified by:
getFileSystem
in interfaceStore
-
getRegionFileSystem
-
getStoreFileTtl
Description copied from interface:StoreConfigInformation
Returns Gets the cf-specific time-to-live for store files.- Specified by:
getStoreFileTtl
in interfaceStoreConfigInformation
-
getMemStoreFlushSize
Description copied from interface:StoreConfigInformation
Returns Gets the Memstore flush size for the region that this store works with.- Specified by:
getMemStoreFlushSize
in interfaceStoreConfigInformation
-
getFlushableSize
- Specified by:
getFlushableSize
in interfaceStore
- Returns:
- The amount of memory we could flush from this memstore; usually this is equal to
Store.getMemStoreSize()
unless we are carrying snapshots and then it will be the size of outstanding snapshots.
-
getSnapshotSize
Description copied from interface:Store
Returns size of the memstore snapshot- Specified by:
getSnapshotSize
in interfaceStore
-
getCompactionCheckMultiplier
- Specified by:
getCompactionCheckMultiplier
in interfaceStoreConfigInformation
- Returns:
- Gets the cf-specific compaction check frequency multiplier. The need for compaction (outside of normal checks during flush, open, etc.) will be ascertained every multiplier * HConstants.THREAD_WAKE_FREQUENCY milliseconds.
-
getBlockingFileCount
Description copied from interface:StoreConfigInformation
The number of files required before flushes for this store will be blocked.- Specified by:
getBlockingFileCount
in interfaceStoreConfigInformation
-
getColumnFamilyDescriptor
- Specified by:
getColumnFamilyDescriptor
in interfaceStore
-
getMaxSequenceId
Description copied from interface:Store
Returns The maximum sequence id in all store files.- Specified by:
getMaxSequenceId
in interfaceStore
-
getMaxMemStoreTS
Description copied from interface:Store
Returns The maximum memstoreTS in all store files.- Specified by:
getMaxMemStoreTS
in interfaceStore
-
getDataBlockEncoder
Returns the data block encoder -
setDataBlockEncoderInTest
Should be used only in tests.- Parameters:
blockEncoder
- the block delta encoder to use
-
postRefreshStoreFiles
- Throws:
IOException
-
refreshStoreFiles
Description copied from interface:Store
Checks the underlying store files, and opens the files that have not been opened, and removes the store file readers for store files no longer available. Mainly used by secondary region replicas to keep up to date with the primary region files.- Specified by:
refreshStoreFiles
in interfaceStore
- Throws:
IOException
-
refreshStoreFiles
Replaces the store files that the store has with the given files. Mainly used by secondary region replicas to keep up to date with the primary region files.- Throws:
IOException
-
startReplayingFromWAL
This message intends to inform the MemStore that next coming updates are going to be part of the replaying edits from WAL -
stopReplayingFromWAL
This message intends to inform the MemStore that the replaying edits from WAL are done -
add
Adds a value to the memstore -
add
Adds the specified value to the memstore -
timeOfOldestEdit
Description copied from interface:Store
When was the last edit done in the memstore- Specified by:
timeOfOldestEdit
in interfaceStore
-
getStorefiles
Returns All store files.- Specified by:
getStorefiles
in interfaceStore
-
getCompactedFiles
- Specified by:
getCompactedFiles
in interfaceStore
-
assertBulkLoadHFileOk
This throws a WrongRegionException if the HFile does not fit in this region, or an InvalidHFileException if the HFile is not valid.- Throws:
IOException
-
preBulkLoadHFile
public Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException This method should only be called from Region. It is assumed that the ranges of values in the HFile fit within the stores assigned region. (assertBulkLoadHFileOk checks this)- Parameters:
seqNum
- sequence Id associated with the HFile- Throws:
IOException
-
bulkLoadHFile
public org.apache.hadoop.fs.Path bulkLoadHFile(byte[] family, String srcPathStr, org.apache.hadoop.fs.Path dstPath) throws IOException - Throws:
IOException
-
bulkLoadHFile
- Throws:
IOException
-
bulkLoadHFile
- Throws:
IOException
-
closeWithoutLock
private org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection<HStoreFile> closeWithoutLock() throws IOException- Throws:
IOException
-
close
public org.apache.hbase.thirdparty.com.google.common.collect.ImmutableCollection<HStoreFile> close() throws IOExceptionClose all the readers We don't need to worry about subsequent requests because the Region holds a write lock that will prevent any more reads or writes.- Returns:
- the
StoreFiles
that were previously being used. - Throws:
IOException
- on failure
-
flushCache
protected List<org.apache.hadoop.fs.Path> flushCache(long logCacheFlushId, MemStoreSnapshot snapshot, MonitoredTask status, ThroughputController throughputController, FlushLifeCycleTracker tracker, Consumer<org.apache.hadoop.fs.Path> writerCreationTracker) throws IOException Write out current snapshot. PresumesStoreFlusherImpl.prepare()
has been called previously.- Parameters:
logCacheFlushId
- flush sequence number- Returns:
- The path name of the tmp file to which the store was flushed
- Throws:
IOException
- if exception occurs during process
-
tryCommitRecoveredHFile
- Throws:
IOException
-
getTotalSize
-
completeFlush
- Throws:
IOException
-
notifyChangedReadersObservers
Notify all observers that set of Readers has changed.- Throws:
IOException
-
getScanners
public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt, boolean onlyLatestVersion) throws IOException Get all scanners with no filtering based on TTL (that happens further down the line).- Parameters:
cacheBlocks
- cache the blocks or notusePread
- true to use pread, false if notisCompaction
- true if the scanner is created for compactionmatcher
- the scan query matcherstartRow
- the start rowstopRow
- the stop rowreadPt
- the read point of the current scan- Returns:
- all scanners for this store
- Throws:
IOException
-
getScanners
public List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean onlyLatestVersion) throws IOException Get all scanners with no filtering based on TTL (that happens further down the line).- Parameters:
cacheBlocks
- cache the blocks or notusePread
- true to use pread, false if notisCompaction
- true if the scanner is created for compactionmatcher
- the scan query matcherstartRow
- the start rowincludeStartRow
- true to include start row, false if notstopRow
- the stop rowincludeStopRow
- true to include stop row, false if notreadPt
- the read point of the current scan- Returns:
- all scanners for this store
- Throws:
IOException
-
clearAndClose
-
getScanners
public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner, boolean onlyLatestVersion) throws IOException Create scanners on the given files and if needed on the memstore with no filtering based on TTL (that happens further down the line).- Parameters:
files
- the list of files on which the scanners has to be createdcacheBlocks
- cache the blocks or notusePread
- true to use pread, false if notisCompaction
- true if the scanner is created for compactionmatcher
- the scan query matcherstartRow
- the start rowstopRow
- the stop rowreadPt
- the read point of the current scanincludeMemstoreScanner
- true if memstore has to be included- Returns:
- scanners on the given files and on the memstore if specified
- Throws:
IOException
-
getScanners
public List<KeyValueScanner> getScanners(List<HStoreFile> files, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner, boolean onlyLatestVersion) throws IOException Create scanners on the given files and if needed on the memstore with no filtering based on TTL (that happens further down the line).- Parameters:
files
- the list of files on which the scanners has to be createdcacheBlocks
- ache the blocks or notusePread
- true to use pread, false if notisCompaction
- true if the scanner is created for compactionmatcher
- the scan query matcherstartRow
- the start rowincludeStartRow
- true to include start row, false if notstopRow
- the stop rowincludeStopRow
- true to include stop row, false if notreadPt
- the read point of the current scanincludeMemstoreScanner
- true if memstore has to be included- Returns:
- scanners on the given files and on the memstore if specified
- Throws:
IOException
-
addChangedReaderObserver
- Parameters:
o
- Observer who wants to know about changes in set of Readers
-
deleteChangedReaderObserver
- Parameters:
o
- Observer no longer interested in changes in set of Readers.
-
compact
public List<HStoreFile> compact(CompactionContext compaction, ThroughputController throughputController, User user) throws IOException Compact the StoreFiles. This method may take some time, so the calling thread must be able to block for long periods.During this time, the Store can work as usual, getting values from StoreFiles and writing new StoreFiles from the memstore. Existing StoreFiles are not destroyed until the new compacted StoreFile is completely written-out to disk.
The compactLock prevents multiple simultaneous compactions. The structureLock prevents us from interfering with other write operations.
We don't want to hold the structureLock for the whole time, as a compact() can be lengthy and we want to allow cache-flushes during this period.
Compaction event should be idempotent, since there is no IO Fencing for the region directory in hdfs. A region server might still try to complete the compaction after it lost the region. That is why the following events are carefully ordered for a compaction: 1. Compaction writes new files under region/.tmp directory (compaction output) 2. Compaction atomically moves the temporary file under region directory 3. Compaction appends a WAL edit containing the compaction input and output files. Forces sync on WAL. 4. Compaction deletes the input files from the region directory. Failure conditions are handled like this: - If RS fails before 2, compaction wont complete. Even if RS lives on and finishes the compaction later, it will only write the new data file to the region directory. Since we already have this data, this will be idempotent but we will have a redundant copy of the data. - If RS fails between 2 and 3, the region will have a redundant copy of the data. The RS that failed won't be able to finish sync() for WAL because of lease recovery in WAL. - If RS fails after 3, the region region server who opens the region will pick up the the compaction marker from the WAL and replay it by removing the compaction input files. Failed RS can also attempt to delete those files, but the operation will be idempotent See HBASE-2231 for details.
- Parameters:
compaction
- compaction details obtained from requestCompaction()- Returns:
- Storefile we compacted into or null if we failed or opted out early.
- Throws:
IOException
-
doCompaction
protected List<HStoreFile> doCompaction(CompactionRequestImpl cr, Collection<HStoreFile> filesToCompact, User user, long compactionStartTime, List<org.apache.hadoop.fs.Path> newFiles) throws IOException - Throws:
IOException
-
setStoragePolicyFromFileName
private void setStoragePolicyFromFileName(List<org.apache.hadoop.fs.Path> newFiles) throws IOException - Throws:
IOException
-
writeCompactionWalRecord
private void writeCompactionWalRecord(Collection<HStoreFile> filesCompacted, Collection<HStoreFile> newFiles) throws IOException Writes the compaction WAL record.- Parameters:
filesCompacted
- Files compacted (input).newFiles
- Files from compaction.- Throws:
IOException
-
replaceStoreFiles
void replaceStoreFiles(Collection<HStoreFile> compactedFiles, Collection<HStoreFile> result, boolean writeCompactionMarker) throws IOException - Throws:
IOException
-
updateSpaceQuotaAfterFileReplacement
void updateSpaceQuotaAfterFileReplacement(RegionSizeStore sizeStore, RegionInfo regionInfo, Collection<HStoreFile> oldFiles, Collection<HStoreFile> newFiles) Updates the space quota usage for this region, removing the size for files compacted away and adding in the size for new files.- Parameters:
sizeStore
- The object tracking changes in region size for space quotas.regionInfo
- The identifier for the region whose size is being updated.oldFiles
- Files removed from this store's region.newFiles
- Files added to this store's region.
-
logCompactionEndMessage
private void logCompactionEndMessage(CompactionRequestImpl cr, List<HStoreFile> sfs, long now, long compactionStartTime) Log a very elaborate compaction completion message.- Parameters:
cr
- Request.sfs
- Resulting files.compactionStartTime
- Start time.
-
replayCompactionMarker
public void replayCompactionMarker(org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor compaction, boolean pickCompactionFiles, boolean removeFiles) throws IOException Call to complete a compaction. Its for the case where we find in the WAL a compaction that was not finished. We could find one recovering a WAL after a regionserver crash. See HBASE-2231.- Throws:
IOException
-
hasReferences
Description copied from interface:Store
Returnstrue
if the store has any underlying reference files to older HFiles- Specified by:
hasReferences
in interfaceStore
-
getCompactionProgress
getter for CompactionProgress object- Returns:
- CompactionProgress object; can be null
-
shouldPerformMajorCompaction
Description copied from interface:Store
Tests whether we should run a major compaction. For example, if the configured major compaction interval is reached.- Specified by:
shouldPerformMajorCompaction
in interfaceStore
- Returns:
- true if we should run a major compaction.
- Throws:
IOException
-
requestCompaction
- Throws:
IOException
-
requestCompaction
public Optional<CompactionContext> requestCompaction(int priority, CompactionLifeCycleTracker tracker, User user) throws IOException - Throws:
IOException
-
addToCompactingFiles
Adds the files to compacting files. filesCompacting must be locked. -
removeUnneededFiles
- Throws:
IOException
-
cancelRequestedCompaction
-
finishCompactionRequest
-
refreshStoreSizeAndTotalBytes
Update counts.- Throws:
IOException
-
versionsToReturn
-
canSplit
Description copied from interface:Store
Returns whether this store is splittable, i.e., no reference file in this store. -
getSplitPoint
Determines if Store should be split. -
getLastCompactSize
Description copied from interface:Store
Returns aggregate size of all HStores used in the last compaction- Specified by:
getLastCompactSize
in interfaceStore
-
getSize
Description copied from interface:Store
Returns aggregate size of HStore -
triggerMajorCompaction
-
recreateScanners
public List<KeyValueScanner> recreateScanners(List<KeyValueScanner> currentFileScanners, boolean cacheBlocks, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, boolean includeStartRow, byte[] stopRow, boolean includeStopRow, long readPt, boolean includeMemstoreScanner) throws IOException Recreates the scanners on the current list of active store file scanners- Parameters:
currentFileScanners
- the current set of active store file scannerscacheBlocks
- cache the blocks or notusePread
- use pread or notisCompaction
- is the scanner for compactionmatcher
- the scan query matcherstartRow
- the scan's start rowincludeStartRow
- should the scan include the start rowstopRow
- the scan's stop rowincludeStopRow
- should the scan include the stop rowreadPt
- the read point of the current scaneincludeMemstoreScanner
- whether the current scanner should include memstorescanner- Returns:
- list of scanners recreated on the current Scanners
- Throws:
IOException
-
toString
-
getStorefilesCount
Description copied from interface:Store
Returns Count of store files- Specified by:
getStorefilesCount
in interfaceStore
-
getCompactedFilesCount
Description copied from interface:Store
Returns Count of compacted store files- Specified by:
getCompactedFilesCount
in interfaceStore
-
getStoreFileAgeStream
-
getMaxStoreFileAge
Description copied from interface:Store
Returns Max age of store files in this store- Specified by:
getMaxStoreFileAge
in interfaceStore
-
getMinStoreFileAge
Description copied from interface:Store
Returns Min age of store files in this store- Specified by:
getMinStoreFileAge
in interfaceStore
-
getAvgStoreFileAge
Description copied from interface:Store
Returns Average age of store files in this store- Specified by:
getAvgStoreFileAge
in interfaceStore
-
getNumReferenceFiles
Description copied from interface:Store
Returns Number of reference files in this store- Specified by:
getNumReferenceFiles
in interfaceStore
-
getNumHFiles
Description copied from interface:Store
Returns Number of HFiles in this store- Specified by:
getNumHFiles
in interfaceStore
-
getStoreSizeUncompressed
Description copied from interface:Store
Returns The size of the store files, in bytes, uncompressed.- Specified by:
getStoreSizeUncompressed
in interfaceStore
-
getStorefilesSize
Description copied from interface:Store
Returns The size of the store files, in bytes.- Specified by:
getStorefilesSize
in interfaceStore
-
getHFilesSize
Description copied from interface:Store
Returns The size of only the store files which are HFiles, in bytes.- Specified by:
getHFilesSize
in interfaceStore
-
getStorefilesFieldSize
-
getStorefilesRootLevelIndexSize
Description copied from interface:Store
Returns The size of the store file root-level indexes, in bytes.- Specified by:
getStorefilesRootLevelIndexSize
in interfaceStore
-
getTotalStaticIndexSize
Description copied from interface:Store
Returns the total size of all index blocks in the data block indexes, including the root level, intermediate levels, and the leaf level for multi-level indexes, or just the root level for single-level indexes.- Specified by:
getTotalStaticIndexSize
in interfaceStore
- Returns:
- the total size of block indexes in the store
-
getTotalStaticBloomSize
Description copied from interface:Store
Returns the total byte size of all Bloom filter bit arrays. For compound Bloom filters even the Bloom blocks currently not loaded into the block cache are counted.- Specified by:
getTotalStaticBloomSize
in interfaceStore
- Returns:
- the total size of all Bloom filters in the store
-
getMemStoreSize
Description copied from interface:Store
Returns The size of this store's memstore.- Specified by:
getMemStoreSize
in interfaceStore
-
getCompactPriority
- Specified by:
getCompactPriority
in interfaceStore
-
throttleCompaction
-
getHRegion
-
getCoprocessorHost
-
getRegionInfo
Description copied from interface:Store
Returns the parent region info hosting this store- Specified by:
getRegionInfo
in interfaceStore
- Specified by:
getRegionInfo
in interfaceStoreConfigInformation
-
areWritesEnabled
- Specified by:
areWritesEnabled
in interfaceStore
-
getSmallestReadPoint
- Specified by:
getSmallestReadPoint
in interfaceStore
- Returns:
- The smallest mvcc readPoint across all the scanners in this region. Writes older than this readPoint, are included in every read operation.
-
upsert
Adds or replaces the specified KeyValues.For each KeyValue specified, if a cell with the same row, family, and qualifier exists in MemStore, it will be replaced. Otherwise, it will just be inserted to MemStore.
This operation is atomic on each KeyValue (row/family/qualifier) but not necessarily atomic across all of them.
- Parameters:
readpoint
- readpoint below which we can safely remove duplicate KVs
-
createFlushContext
-
needsCompaction
Description copied from interface:Store
See if there's too much store files in this store- Specified by:
needsCompaction
in interfaceStore
- Returns:
true
if number of store files is greater than the number defined in minFilesToCompact
-
getCacheConfig
Used for tests.- Returns:
- cache configuration for this Store.
-
heapSize
Description copied from interface:HeapSize
Return the approximate 'exclusive deep size' of implementing object. Includes count of payload and hosting object sizings. -
getComparator
- Specified by:
getComparator
in interfaceStore
-
getScanInfo
-
setScanInfo
Set scan info, used by test- Parameters:
scanInfo
- new scan info to use for test
-
hasTooManyStoreFiles
Description copied from interface:Store
Returns Whether this store has too many store files.- Specified by:
hasTooManyStoreFiles
in interfaceStore
-
getFlushedCellsCount
Description copied from interface:Store
Returns The number of cells flushed to disk- Specified by:
getFlushedCellsCount
in interfaceStore
-
getFlushedCellsSize
Description copied from interface:Store
Returns The total size of data flushed to disk, in bytes- Specified by:
getFlushedCellsSize
in interfaceStore
-
getFlushedOutputFileSize
Description copied from interface:Store
Returns The total size of out output files on disk, in bytes- Specified by:
getFlushedOutputFileSize
in interfaceStore
-
getCompactedCellsCount
Description copied from interface:Store
Returns The number of cells processed during minor compactions- Specified by:
getCompactedCellsCount
in interfaceStore
-
getCompactedCellsSize
Description copied from interface:Store
Returns The total amount of data processed during minor compactions, in bytes- Specified by:
getCompactedCellsSize
in interfaceStore
-
getMajorCompactedCellsCount
Description copied from interface:Store
Returns The number of cells processed during major compactions- Specified by:
getMajorCompactedCellsCount
in interfaceStore
-
getMajorCompactedCellsSize
Description copied from interface:Store
Returns The total amount of data processed during major compactions, in bytes- Specified by:
getMajorCompactedCellsSize
in interfaceStore
-
updateCompactedMetrics
-
getStoreEngine
Returns the StoreEngine that is backing this concrete implementation of Store.- Returns:
- Returns the
StoreEngine
object used internally inside this HStore object.
-
getOffPeakHours
-
onConfigurationChange
Description copied from interface:ConfigurationObserver
This method would be called by theConfigurationManager
object when theConfiguration
object is reloaded from disk.- Specified by:
onConfigurationChange
in interfaceConfigurationObserver
-
registerChildren
Needs to be called to register the children to the manager.- Specified by:
registerChildren
in interfacePropagatingConfigurationObserver
- Parameters:
manager
- : to register to
-
deregisterChildren
Needs to be called to deregister the children from the manager.- Specified by:
deregisterChildren
in interfacePropagatingConfigurationObserver
- Parameters:
manager
- : to deregister from
-
getCompactionPressure
Description copied from interface:Store
This value can represent the degree of emergency of compaction for this store. It should be greater than or equal to 0.0, any value greater than 1.0 means we have too many store files.- if getStorefilesCount <= getMinFilesToCompact, return 0.0
- return (getStorefilesCount - getMinFilesToCompact) / (blockingFileCount - getMinFilesToCompact)
And for striped stores, we should calculate this value by the files in each stripe separately and return the maximum value.
It is similar to
Store.getCompactPriority()
except that it is more suitable to use in a linear formula.- Specified by:
getCompactionPressure
in interfaceStore
-
isPrimaryReplicaStore
- Specified by:
isPrimaryReplicaStore
in interfaceStore
-
preSnapshotOperation
Sets the store up for a region level snapshot operation.- See Also:
-
postSnapshotOperation
Perform tasks needed after the completion of snapshot operation.- See Also:
-
closeAndArchiveCompactedFiles
Closes and archives the compacted files under this store- Throws:
IOException
-
removeCompactedfiles
private void removeCompactedfiles(Collection<HStoreFile> compactedfiles, boolean evictOnClose) throws IOException Archives and removes the compacted files- Parameters:
compactedfiles
- The compacted files in this store that are not active in readsevictOnClose
- true if blocks should be evicted from the cache when an HFile reader is closed, false if not- Throws:
IOException
-
getStoreFileSize
Computes the length of a store file without succumbing to any errors along the way. If an error is encountered, the implementation returns0
instead of the actual size.- Parameters:
file
- The file to compute the size of.- Returns:
- The size in bytes of the provided
file
.
-
preFlushSeqIDEstimation
-
isSloppyMemStore
Description copied from interface:Store
Returns true if the memstore may need some extra memory space- Specified by:
isSloppyMemStore
in interfaceStore
-
clearCompactedfiles
- Throws:
IOException
-
reportArchivedFilesForQuota
-
getCurrentParallelPutCount
- Specified by:
getCurrentParallelPutCount
in interfaceStore
-
getStoreRefCount
-
getMaxCompactedStoreFileRefCount
Returns get maximum ref count of storeFile among all compacted HStore Files for the HStore -
getMemstoreOnlyRowReadsCount
Description copied from interface:Store
Returns the number of read requests purely from the memstore.- Specified by:
getMemstoreOnlyRowReadsCount
in interfaceStore
-
getMixedRowReadsCount
Description copied from interface:Store
Returns the number of read requests from the files under this store.- Specified by:
getMixedRowReadsCount
in interfaceStore
-
getReadOnlyConfiguration
- Specified by:
getReadOnlyConfiguration
in interfaceStore
- Returns:
- a read only configuration of this store; throws
UnsupportedOperationException
if you try to set a configuration.
-
updateMetricsStore
-
getStoreFilesBeingWritten
Return the storefiles which are currently being written to. Mainly used byBrokenStoreFileCleaner
to prevent deleting the these files as they are not present in SFT yet. -
getBloomFilterRequestsCount
Description copied from interface:Store
Returns count of bloom filter results for this store.- Specified by:
getBloomFilterRequestsCount
in interfaceStore
-
getBloomFilterNegativeResultsCount
Description copied from interface:Store
Returns count of negative results for bloom filter requests for this store.- Specified by:
getBloomFilterNegativeResultsCount
in interfaceStore
-
getBloomFilterEligibleRequestsCount
Description copied from interface:Store
Returns count of requests which could have used bloom filters, but they weren't configured or loaded.- Specified by:
getBloomFilterEligibleRequestsCount
in interfaceStore
-