@InterfaceAudience.Private public class HStore extends Object implements Store
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.
| Modifier and Type | Field and Description |
|---|---|
static String |
BLOCKING_STOREFILES_KEY |
static String |
COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY |
static long |
DEEP_OVERHEAD |
static int |
DEFAULT_BLOCKING_STOREFILE_COUNT |
static int |
DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER |
static long |
FIXED_OVERHEAD |
protected MemStore |
memstore |
NO_PRIORITY, PRIORITY_USER| Modifier | Constructor and Description |
|---|---|
protected |
HStore(HRegion region,
HColumnDescriptor family,
org.apache.hadoop.conf.Configuration confParam)
Constructor
|
| Modifier and Type | Method and Description |
|---|---|
long |
add(Cell cell)
Adds a value to the memstore
|
long |
add(Iterable<Cell> cells)
Adds the specified value to the memstore
|
void |
addChangedReaderObserver(ChangedReadersObserver o) |
boolean |
areWritesEnabled() |
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)
This method should only be called from Region.
|
void |
bulkLoadHFile(StoreFileInfo fileInfo) |
void |
cancelRequestedCompaction(CompactionContext compaction) |
boolean |
canSplit() |
com.google.common.collect.ImmutableCollection<StoreFile> |
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 |
closeAndArchiveCompactedFiles()
Closes and archives the compacted files under this store
|
List<StoreFile> |
compact(CompactionContext compaction,
ThroughputController throughputController)
Compact the StoreFiles.
|
List<StoreFile> |
compact(CompactionContext compaction,
ThroughputController throughputController,
User user) |
void |
compactRecentForTestingAssumingDefaultPolicy(int N)
This method tries to compact N recent files for testing.
|
protected void |
completeCompaction(Collection<StoreFile> compactedFiles)
It works by processing a compaction that's been written to disk.
|
protected void |
completeCompaction(Collection<StoreFile> compactedFiles,
boolean removeFiles)
It works by processing a compaction that's been written to disk.
|
org.apache.hadoop.hbase.regionserver.StoreFlushContext |
createFlushContext(long cacheFlushId) |
StoreFile.Writer |
createWriterInTmp(long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTag) |
StoreFile.Writer |
createWriterInTmp(long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTag,
boolean shouldDropBehind) |
StoreFile.Writer |
createWriterInTmp(long maxKeyCount,
Compression.Algorithm compression,
boolean isCompaction,
boolean includeMVCCReadpoint,
boolean includesTag,
boolean shouldDropBehind,
TimeRangeTracker trt) |
protected long |
delete(KeyValue kv)
Adds a value to the memstore
|
void |
deleteChangedReaderObserver(ChangedReadersObserver o) |
void |
deregisterChildren(ConfigurationManager manager)
Needs to be called to deregister the children from the manager.
|
protected List<org.apache.hadoop.fs.Path> |
flushCache(long logCacheFlushId,
MemStoreSnapshot snapshot,
MonitoredTask status,
ThroughputController throughputController)
Write out current snapshot.
|
long |
getAvgStoreFileAge() |
long |
getBlockingFileCount()
The number of files required before flushes for this store will be blocked.
|
static int |
getBytesPerChecksum(org.apache.hadoop.conf.Configuration conf)
Returns the configured bytesPerChecksum value.
|
CacheConfig |
getCacheConfig()
Used for tests.
|
static ChecksumType |
getChecksumType(org.apache.hadoop.conf.Configuration conf)
Returns the configured checksum algorithm.
|
static int |
getCloseCheckInterval() |
String |
getColumnFamilyName() |
long |
getCompactedCellsCount() |
long |
getCompactedCellsSize() |
Collection<StoreFile> |
getCompactedfiles() |
long |
getCompactionCheckMultiplier() |
double |
getCompactionPressure()
This value can represent the degree of emergency of compaction for this store.
|
CompactionProgress |
getCompactionProgress()
getter for CompactionProgress object
|
int |
getCompactPriority() |
KeyValue.KVComparator |
getComparator() |
RegionCoprocessorHost |
getCoprocessorHost() |
HFileDataBlockEncoder |
getDataBlockEncoder() |
HColumnDescriptor |
getFamily() |
org.apache.hadoop.fs.FileSystem |
getFileSystem() |
long |
getFlushableSize() |
long |
getFlushedCellsCount() |
long |
getFlushedCellsSize() |
long |
getFlushedOutputFileSize() |
HRegion |
getHRegion() |
long |
getLastCompactSize() |
long |
getMajorCompactedCellsCount() |
long |
getMajorCompactedCellsSize() |
long |
getMaxMemstoreTS() |
long |
getMaxSequenceId() |
long |
getMaxStoreFileAge() |
long |
getMemstoreFlushSize() |
long |
getMemStoreSize() |
long |
getMinStoreFileAge() |
long |
getNumHFiles() |
long |
getNumReferenceFiles() |
protected OffPeakHours |
getOffPeakHours() |
HRegionFileSystem |
getRegionFileSystem() |
HRegionInfo |
getRegionInfo() |
Cell |
getRowKeyAtOrBefore(byte[] row)
Find the key that matches row exactly, or the one that immediately precedes it.
|
ScanInfo |
getScanInfo() |
KeyValueScanner |
getScanner(Scan scan,
NavigableSet<byte[]> targetCols,
long readPt)
Return a scanner for both the memstore and the HStore files.
|
List<KeyValueScanner> |
getScanners(boolean cacheBlocks,
boolean isGet,
boolean usePread,
boolean isCompaction,
ScanQueryMatcher matcher,
byte[] startRow,
byte[] stopRow,
long readPt)
Get all scanners with no filtering based on TTL (that happens further down
the line).
|
List<KeyValueScanner> |
getScanners(List<StoreFile> files,
boolean cacheBlocks,
boolean isGet,
boolean usePread,
boolean isCompaction,
ScanQueryMatcher matcher,
byte[] startRow,
byte[] stopRow,
long readPt,
boolean includeMemstoreScanner)
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() |
long |
getSmallestReadPoint() |
long |
getSnapshotSize()
Returns the memstore snapshot size
|
byte[] |
getSplitPoint()
Determines if Store should be split
|
StoreEngine<?,?,?,?> |
getStoreEngine()
Returns the StoreEngine that is backing this concrete implementation of Store.
|
Collection<StoreFile> |
getStorefiles() |
int |
getStorefilesCount() |
long |
getStorefilesIndexSize() |
long |
getStorefilesSize() |
long |
getStoreFileTtl() |
static org.apache.hadoop.fs.Path |
getStoreHomedir(org.apache.hadoop.fs.Path tabledir,
HRegionInfo hri,
byte[] family)
Deprecated.
|
static org.apache.hadoop.fs.Path |
getStoreHomedir(org.apache.hadoop.fs.Path tabledir,
String encodedName,
byte[] family)
Deprecated.
|
long |
getStoreSizeUncompressed() |
TableName |
getTableName() |
long |
getTotalStaticBloomSize()
Returns the total byte size of all Bloom filter bit arrays.
|
long |
getTotalStaticIndexSize()
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 |
hasReferences() |
boolean |
hasTooManyStoreFiles() |
long |
heapSize() |
static boolean |
isCellTTLExpired(Cell cell,
long oldestTimestamp,
long now) |
boolean |
isMajorCompaction() |
boolean |
isPrimaryReplicaStore() |
boolean |
needsCompaction()
See if there's too much store files in this store
|
void |
onConfigurationChange(org.apache.hadoop.conf.Configuration conf)
This method would be called by the
ConfigurationManager
object when the Configuration object is reloaded from disk. |
void |
postSnapshotOperation()
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) |
void |
preSnapshotOperation()
Sets the store up for a region level snapshot operation.
|
void |
refreshStoreFiles()
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.
|
void |
registerChildren(ConfigurationManager manager)
Needs to be called to register the children to the manager.
|
void |
replayCompactionMarker(org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor compaction,
boolean pickCompactionFiles,
boolean removeFiles)
Call to complete a compaction.
|
CompactionContext |
requestCompaction() |
CompactionContext |
requestCompaction(int priority,
CompactionRequest baseRequest) |
CompactionContext |
requestCompaction(int priority,
CompactionRequest baseRequest,
User user) |
void |
rollback(Cell cell)
Removes a Cell from the memstore.
|
boolean |
throttleCompaction(long compactionSize) |
long |
timeOfOldestEdit()
When was the last edit done in the memstore
|
String |
toString() |
void |
triggerMajorCompaction() |
long |
updateColumnValue(byte[] row,
byte[] f,
byte[] qualifier,
long newValue)
Used in tests.
|
long |
upsert(Iterable<Cell> cells,
long readpoint,
List<Cell> removedCells)
Adds or replaces the specified KeyValues.
|
public static final String COMPACTCHECKER_INTERVAL_MULTIPLIER_KEY
public static final String BLOCKING_STOREFILES_KEY
public static final int DEFAULT_COMPACTCHECKER_INTERVAL_MULTIPLIER
public static final int DEFAULT_BLOCKING_STOREFILE_COUNT
protected final MemStore memstore
public static final long FIXED_OVERHEAD
public static final long DEEP_OVERHEAD
protected HStore(HRegion region, HColumnDescriptor family, org.apache.hadoop.conf.Configuration confParam) throws IOException
region - family - HColumnDescriptor for this columnconfParam - configuration object
failed. Can be null.IOExceptionpublic String getColumnFamilyName()
getColumnFamilyName in interface Storepublic TableName getTableName()
getTableName in interface Storepublic org.apache.hadoop.fs.FileSystem getFileSystem()
getFileSystem in interface Storepublic HRegionFileSystem getRegionFileSystem()
public long getStoreFileTtl()
getStoreFileTtl in interface StoreConfigInformationpublic long getMemstoreFlushSize()
getMemstoreFlushSize in interface StoreConfigInformationpublic long getFlushableSize()
getFlushableSize in interface StoreStore.getMemStoreSize() unless we are carrying snapshots and then it will be the size of
outstanding snapshots.public long getSnapshotSize()
StoregetSnapshotSize in interface Storepublic long getCompactionCheckMultiplier()
getCompactionCheckMultiplier in interface StoreConfigInformationpublic long getBlockingFileCount()
StoreConfigInformationgetBlockingFileCount in interface StoreConfigInformationpublic static int getBytesPerChecksum(org.apache.hadoop.conf.Configuration conf)
conf - The configurationpublic static ChecksumType getChecksumType(org.apache.hadoop.conf.Configuration conf)
conf - The configurationpublic static int getCloseCheckInterval()
public HColumnDescriptor getFamily()
public long getMaxSequenceId()
getMaxSequenceId in interface Storepublic long getMaxMemstoreTS()
getMaxMemstoreTS in interface Store@Deprecated public static org.apache.hadoop.fs.Path getStoreHomedir(org.apache.hadoop.fs.Path tabledir, HRegionInfo hri, byte[] family)
tabledir - Path to where the table is being storedhri - HRegionInfo for the region.family - HColumnDescriptor describing the column family@Deprecated public static org.apache.hadoop.fs.Path getStoreHomedir(org.apache.hadoop.fs.Path tabledir, String encodedName, byte[] family)
tabledir - Path to where the table is being storedencodedName - Encoded region name.family - HColumnDescriptor describing the column familypublic HFileDataBlockEncoder getDataBlockEncoder()
getDataBlockEncoder in interface Storepublic void refreshStoreFiles()
throws IOException
refreshStoreFiles in interface StoreIOExceptionpublic void refreshStoreFiles(Collection<String> newFiles) throws IOException
StorerefreshStoreFiles in interface StoreIOExceptionpublic long add(Cell cell)
Storepublic long add(Iterable<Cell> cells)
Storepublic long timeOfOldestEdit()
StoretimeOfOldestEdit in interface Storeprotected long delete(KeyValue kv)
kv - public void rollback(Cell cell)
Storepublic Collection<StoreFile> getStorefiles()
getStorefiles in interface Storepublic Collection<StoreFile> getCompactedfiles()
public void assertBulkLoadHFileOk(org.apache.hadoop.fs.Path srcPath)
throws IOException
StoreassertBulkLoadHFileOk in interface StoreIOExceptionpublic Pair<org.apache.hadoop.fs.Path,org.apache.hadoop.fs.Path> preBulkLoadHFile(String srcPathStr, long seqNum) throws IOException
IOExceptionpublic org.apache.hadoop.fs.Path bulkLoadHFile(byte[] family,
String srcPathStr,
org.apache.hadoop.fs.Path dstPath)
throws IOException
StorebulkLoadHFile in interface Storefamily - the column familyIOExceptionpublic void bulkLoadHFile(StoreFileInfo fileInfo) throws IOException
bulkLoadHFile in interface StoreIOExceptionpublic com.google.common.collect.ImmutableCollection<StoreFile> close() throws IOException
Storeclose in interface StoreStoreFiles that were previously being used.IOException - on failureprotected List<org.apache.hadoop.fs.Path> flushCache(long logCacheFlushId, MemStoreSnapshot snapshot, MonitoredTask status, ThroughputController throughputController) throws IOException
snapshot() has been called previously.logCacheFlushId - flush sequence numbersnapshot - status - throughputController - IOException - if exception occurs during processpublic StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag) throws IOException
createWriterInTmp in interface Storecompression - Compression algorithm to useisCompaction - whether we are creating a new file in a compactionincludeMVCCReadpoint - whether we should out the MVCC readpointIOExceptionpublic StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, boolean shouldDropBehind) throws IOException
createWriterInTmp in interface Storecompression - Compression algorithm to useisCompaction - whether we are creating a new file in a compactionincludeMVCCReadpoint - whether we should out the MVCC readpointshouldDropBehind - should the writer drop caches behind writesIOExceptionpublic StoreFile.Writer createWriterInTmp(long maxKeyCount, Compression.Algorithm compression, boolean isCompaction, boolean includeMVCCReadpoint, boolean includesTag, boolean shouldDropBehind, TimeRangeTracker trt) throws IOException
createWriterInTmp in interface Storecompression - Compression algorithm to useisCompaction - whether we are creating a new file in a compactionincludeMVCCReadpoint - whether we should out the MVCC readpointshouldDropBehind - should the writer drop caches behind writestrt - Ready-made timetracker to use.IOExceptionpublic List<KeyValueScanner> getScanners(boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt) throws IOException
getScanners in interface StoreIOExceptionpublic List<KeyValueScanner> getScanners(List<StoreFile> files, boolean cacheBlocks, boolean isGet, boolean usePread, boolean isCompaction, ScanQueryMatcher matcher, byte[] startRow, byte[] stopRow, long readPt, boolean includeMemstoreScanner) throws IOException
StoregetScanners in interface Storefiles - the list of files on which the scanners has to be createdcacheBlocks - cache the blocks or notisGet - true if it is get, false if 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 includedIOExceptionpublic void addChangedReaderObserver(ChangedReadersObserver o)
addChangedReaderObserver in interface Storepublic void deleteChangedReaderObserver(ChangedReadersObserver o)
deleteChangedReaderObserver in interface Storepublic List<StoreFile> compact(CompactionContext compaction, ThroughputController throughputController) throws IOException
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 snyc() 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.
compact in interface Storecompaction - compaction details obtained from requestCompaction()IOExceptionpublic List<StoreFile> compact(CompactionContext compaction, ThroughputController throughputController, User user) throws IOException
compact in interface StoreIOExceptionpublic void replayCompactionMarker(org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor compaction,
boolean pickCompactionFiles,
boolean removeFiles)
throws IOException
replayCompactionMarker in interface Storecompaction - pickCompactionFiles - whether or not pick up the new compaction output files and
add it to the storeremoveFiles - whether to remove/archive files from filesystemIOExceptionpublic void compactRecentForTestingAssumingDefaultPolicy(int N)
throws IOException
N - Number of files.IOExceptionpublic boolean hasReferences()
hasReferences in interface Storepublic CompactionProgress getCompactionProgress()
StoregetCompactionProgress in interface Storepublic boolean isMajorCompaction()
throws IOException
isMajorCompaction in interface StoreIOExceptionpublic CompactionContext requestCompaction() throws IOException
requestCompaction in interface StoreIOExceptionpublic CompactionContext requestCompaction(int priority, CompactionRequest baseRequest) throws IOException
requestCompaction in interface StoreIOExceptionpublic CompactionContext requestCompaction(int priority, CompactionRequest baseRequest, User user) throws IOException
requestCompaction in interface StoreIOExceptionpublic void cancelRequestedCompaction(CompactionContext compaction)
cancelRequestedCompaction in interface Storeprotected void completeCompaction(Collection<StoreFile> compactedFiles) throws IOException
It works by processing a compaction that's been written to disk.
It is usually invoked at the end of a compaction, but might also be invoked at HStore startup, if the prior execution died midway through.
Moving the compacted TreeMap into place means:
1) Unload all replaced StoreFile, close and collect list to delete. 2) Compute new store size
compactedFiles - list of files that were compactedIOExceptionprotected void completeCompaction(Collection<StoreFile> compactedFiles, boolean removeFiles) throws IOException
It works by processing a compaction that's been written to disk.
It is usually invoked at the end of a compaction, but might also be invoked at HStore startup, if the prior execution died midway through.
Moving the compacted TreeMap into place means:
1) Unload all replaced StoreFile, close and collect list to delete. 2) Compute new store size
compactedFiles - list of files that were compactedIOExceptionpublic static boolean isCellTTLExpired(Cell cell, long oldestTimestamp, long now)
cell - oldestTimestamp - public Cell getRowKeyAtOrBefore(byte[] row) throws IOException
StoregetRowKeyAtOrBefore in interface Storerow - The row key of the targeted row.IOExceptionpublic byte[] getSplitPoint()
StoregetSplitPoint in interface Storepublic long getLastCompactSize()
getLastCompactSize in interface Storepublic long getSize()
public void triggerMajorCompaction()
triggerMajorCompaction in interface Storepublic KeyValueScanner getScanner(Scan scan, NavigableSet<byte[]> targetCols, long readPt) throws IOException
StoregetScanner in interface Storescan - Scan to apply when scanning the storestargetCols - columns to scanIOException - on failurepublic int getStorefilesCount()
getStorefilesCount in interface Storepublic long getMaxStoreFileAge()
getMaxStoreFileAge in interface Storepublic long getMinStoreFileAge()
getMinStoreFileAge in interface Storepublic long getAvgStoreFileAge()
getAvgStoreFileAge in interface Storepublic long getNumReferenceFiles()
getNumReferenceFiles in interface Storepublic long getNumHFiles()
getNumHFiles in interface Storepublic long getStoreSizeUncompressed()
getStoreSizeUncompressed in interface Storepublic long getStorefilesSize()
getStorefilesSize in interface Storepublic long getStorefilesIndexSize()
getStorefilesIndexSize in interface Storepublic long getTotalStaticIndexSize()
StoregetTotalStaticIndexSize in interface Storepublic long getTotalStaticBloomSize()
StoregetTotalStaticBloomSize in interface Storepublic long getMemStoreSize()
getMemStoreSize in interface Storepublic int getCompactPriority()
getCompactPriority in interface Storepublic boolean throttleCompaction(long compactionSize)
throttleCompaction in interface Storepublic HRegion getHRegion()
public RegionCoprocessorHost getCoprocessorHost()
getCoprocessorHost in interface Storepublic HRegionInfo getRegionInfo()
getRegionInfo in interface Storepublic boolean areWritesEnabled()
areWritesEnabled in interface Storepublic long getSmallestReadPoint()
getSmallestReadPoint in interface Storepublic long updateColumnValue(byte[] row,
byte[] f,
byte[] qualifier,
long newValue)
throws IOException
row - row to updatef - family to updatequalifier - qualifier to updatenewValue - the new value to set into memstoreIOExceptionpublic long upsert(Iterable<Cell> cells, long readpoint, List<Cell> removedCells) throws IOException
StoreFor 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.
upsert in interface Storereadpoint - readpoint below which we can safely remove duplicate KVsremovedCells - collect the removed cells. It can be null.IOExceptionpublic org.apache.hadoop.hbase.regionserver.StoreFlushContext createFlushContext(long cacheFlushId)
createFlushContext in interface Storepublic boolean needsCompaction()
StoreneedsCompaction in interface Storepublic CacheConfig getCacheConfig()
StoregetCacheConfig in interface Storepublic long heapSize()
public KeyValue.KVComparator getComparator()
getComparator in interface Storepublic ScanInfo getScanInfo()
getScanInfo in interface Storepublic boolean hasTooManyStoreFiles()
hasTooManyStoreFiles in interface Storepublic long getFlushedCellsCount()
getFlushedCellsCount in interface Storepublic long getFlushedCellsSize()
getFlushedCellsSize in interface Storepublic long getFlushedOutputFileSize()
getFlushedOutputFileSize in interface Storepublic long getCompactedCellsCount()
getCompactedCellsCount in interface Storepublic long getCompactedCellsSize()
getCompactedCellsSize in interface Storepublic long getMajorCompactedCellsCount()
getMajorCompactedCellsCount in interface Storepublic long getMajorCompactedCellsSize()
getMajorCompactedCellsSize in interface Storepublic StoreEngine<?,?,?,?> getStoreEngine()
StoreEngine object used internally inside this HStore object.protected OffPeakHours getOffPeakHours()
public void onConfigurationChange(org.apache.hadoop.conf.Configuration conf)
ConfigurationManager
object when the Configuration object is reloaded from disk.onConfigurationChange in interface ConfigurationObserverpublic void registerChildren(ConfigurationManager manager)
registerChildren in interface PropagatingConfigurationObservermanager - : to register topublic void deregisterChildren(ConfigurationManager manager)
deregisterChildren in interface PropagatingConfigurationObservermanager - : to deregister frompublic double getCompactionPressure()
StoreAnd 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.
getCompactionPressure in interface Storepublic boolean isPrimaryReplicaStore()
isPrimaryReplicaStore in interface Storepublic void preSnapshotOperation()
postSnapshotOperation()public void postSnapshotOperation()
preSnapshotOperation()public void closeAndArchiveCompactedFiles()
throws IOException
StorecloseAndArchiveCompactedFiles in interface StoreIOExceptionCopyright © 2007–2019 The Apache Software Foundation. All rights reserved.