public abstract class SSTableReader extends SSTable implements SelfRefCounted<SSTableReader>
NORMAL
From: None => Reader has been read from disk, either at startup or from a flushed memtable
EARLY => Reader is the final result of a compaction
MOVED_START => Reader WAS being compacted, but this failed and it has been restored to NORMAL status
EARLY
From: None => Reader is a compaction replacement that is either incomplete and has been opened
to represent its partial result status, or has been finished but the compaction
it is a part of has not yet completed fully
EARLY => Same as from None, only it is not the first time it has been
MOVED_START
From: NORMAL => Reader is being compacted. This compaction has not finished, but the compaction result
is either partially or fully opened, to either partially or fully replace this reader.
This reader's start key has been updated to represent this, so that reads only hit
one or the other reader.
METADATA_CHANGE
From: NORMAL => Reader has seen low traffic and the amount of memory available for index summaries is
constrained, so its index summary has been downsampled.
METADATA_CHANGE => Same
Note that in parallel to this, there are two different Descriptor types; TMPLINK and FINAL; the latter corresponds
to NORMAL state readers and all readers that replace a NORMAL one. TMPLINK is used for EARLY state readers and
no others.
When a reader is being compacted, if the result is large its replacement may be opened as EARLY before compaction
completes in order to present the result to consumers earlier. In this case the reader will itself be changed to
a MOVED_START state, where its start no longer represents its on-disk minimum key. This is to permit reads to be
directed to only one reader when the two represent the same data. The EARLY file can represent a compaction result
that is either partially complete and still in-progress, or a complete and immutable sstable that is part of a larger
macro compaction action that has not yet fully completed.
Currently ALL compaction results at least briefly go through an EARLY open state prior to completion, regardless
of if early opening is enabled.
Since a reader can be created multiple times over the same shared underlying resources, and the exact resources
it shares between each instance differ subtly, we track the lifetime of any underlying resource with its own
reference count, which each instance takes a Ref to. Each instance then tracks references to itself, and once these
all expire it releases its Refs to these underlying resources.
There is some shared cleanup behaviour needed only once all sstablereaders in a certain stage of their lifecycle
(i.e. EARLY or NORMAL opening), and some that must only occur once all readers of any kind over a single logical
sstable have expired. These are managed by the TypeTidy and GlobalTidy classes at the bottom, and are effectively
managed as another resource each instance tracks its own Ref instance to, to ensure all of these resources are
cleaned up safely and can be debugged otherwise.
TODO: fill in details about Tracker and lifecycle interactions for tools, and for compaction strategiesModifier and Type | Class and Description |
---|---|
static class |
SSTableReader.Factory |
static class |
SSTableReader.IndexesBounds |
static class |
SSTableReader.OpenReason |
static class |
SSTableReader.Operator
TODO: Move someplace reusable
|
static class |
SSTableReader.PartitionPositionBounds |
static class |
SSTableReader.UniqueIdentifier |
RefCounted.Tidy
components, compression, descriptor, first, last, metadata, optimizationStrategy, TOMBSTONE_HISTOGRAM_BIN_SIZE, TOMBSTONE_HISTOGRAM_SPOOL_SIZE, TOMBSTONE_HISTOGRAM_TTL_ROUND_SECONDS
Modifier | Constructor and Description |
---|---|
protected |
SSTableReader(Descriptor desc,
java.util.Set<Component> components,
TableMetadataRef metadata,
long maxDataAge,
StatsMetadata sstableMetadata,
SSTableReader.OpenReason openReason,
SerializationHeader header,
IndexSummary summary,
FileHandle dfile,
FileHandle ifile,
IFilter bf) |
protected |
SSTableReader(SSTableReaderBuilder builder) |
Modifier and Type | Method and Description |
---|---|
void |
addTo(Ref.IdentityCollection identities) |
void |
cacheKey(DecoratedKey key,
RowIndexEntry info) |
SSTableReader |
cloneAndReplace(IFilter newBloomFilter)
Clone this reader with the new values and set the clone as replacement.
|
SSTableReader |
cloneWithNewStart(DecoratedKey newStart,
java.lang.Runnable runOnClose) |
SSTableReader |
cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent,
int samplingLevel)
Returns a new SSTableReader with the same properties as this SSTableReader except that a new IndexSummary will
be built at the target samplingLevel.
|
SSTableReader |
cloneWithRestoredStart(DecoratedKey restoredStart) |
static void |
createLinks(Descriptor descriptor,
java.util.Set<Component> components,
java.lang.String snapshotDirectoryPath) |
static void |
createLinks(Descriptor descriptor,
java.util.Set<Component> components,
java.lang.String snapshotDirectoryPath,
com.google.common.util.concurrent.RateLimiter limiter) |
void |
createLinks(java.lang.String snapshotDirectoryPath) |
void |
createLinks(java.lang.String snapshotDirectoryPath,
com.google.common.util.concurrent.RateLimiter rateLimiter) |
boolean |
equals(java.lang.Object that) |
static double |
estimateCompactionGain(java.util.Set<SSTableReader> overlapping)
Estimates how much of the keys we would keep if the sstables were compacted together
|
long |
estimatedKeys() |
long |
estimatedKeysForRanges(java.util.Collection<Range<Token>> ranges) |
DecoratedKey |
firstKeyBeyond(PartitionPosition token)
Finds and returns the first key beyond a given token in this SSTable or null if no such key exists.
|
static long |
getApproximateKeyCount(java.lang.Iterable<SSTableReader> sstables)
Calculate approximate key count.
|
int |
getAvgColumnSetPerRow() |
IFilter |
getBloomFilter() |
long |
getBloomFilterFalsePositiveCount() |
long |
getBloomFilterOffHeapSize()
Returns the amount of memory in bytes used off heap by the bloom filter.
|
long |
getBloomFilterSerializedSize() |
long |
getBloomFilterTrueNegativeCount() |
long |
getBloomFilterTruePositiveCount() |
RowIndexEntry |
getCachedPosition(DecoratedKey key,
boolean updateStats) |
protected RowIndexEntry |
getCachedPosition(KeyCacheKey unifiedKey,
boolean updateStats) |
KeyCacheKey |
getCacheKey(DecoratedKey key) |
CompressionMetadata |
getCompressionMetadata()
Returns the compression metadata for this sstable.
|
long |
getCompressionMetadataOffHeapSize()
Returns the amount of memory in bytes used off heap by the compression meta-data.
|
double |
getCompressionRatio() |
double |
getCrcCheckChance() |
long |
getCreationTimeFor(Component component) |
ChannelProxy |
getDataChannel() |
double |
getDroppableTombstonesBefore(int gcBefore) |
double |
getEffectiveIndexInterval() |
EstimatedHistogram |
getEstimatedCellPerPartitionCount() |
double |
getEstimatedDroppableTombstoneRatio(int gcBefore) |
EstimatedHistogram |
getEstimatedPartitionSize() |
FileDataInput |
getFileDataInput(long position) |
java.lang.String |
getFilename() |
ChannelProxy |
getIndexChannel() |
FileHandle |
getIndexFile() |
long |
getIndexScanPosition(PartitionPosition key)
Gets the position in the index file to start scanning to find the given key (at most indexInterval keys away,
modulo downsampling of the index summary).
|
static long |
getIndexScanPositionFromBinarySearchResult(int binarySearchResult,
IndexSummary referencedIndexSummary) |
static int |
getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult) |
byte[] |
getIndexSummaryKey(int index)
Returns the key for the index summary entry at `index`.
|
long |
getIndexSummaryOffHeapSize() |
int |
getIndexSummarySamplingLevel() |
int |
getIndexSummarySize()
Returns the number of entries in the IndexSummary.
|
InstrumentingCache<KeyCacheKey,RowIndexEntry> |
getKeyCache() |
long |
getKeyCacheHit() |
long |
getKeyCacheRequest() |
java.lang.Iterable<DecoratedKey> |
getKeySamples(Range<Token> range) |
int |
getMaxIndexSummarySize()
Returns the approximate number of entries the IndexSummary would contain if it were at full sampling.
|
int |
getMaxLocalDeletionTime() |
long |
getMaxTimestamp() |
int |
getMaxTTL() |
int |
getMinIndexInterval() |
int |
getMinLocalDeletionTime() |
long |
getMinTimestamp() |
int |
getMinTTL() |
java.util.UUID |
getPendingRepair() |
RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op)
Retrieves the position while updating the key cache and the stats.
|
RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op,
boolean updateCacheAndStats) |
protected abstract RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op,
boolean updateCacheAndStats,
boolean permitMatchPastLast,
SSTableReadsListener listener) |
RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op,
SSTableReadsListener listener)
Retrieves the position while updating the key cache and the stats.
|
java.util.List<SSTableReader.PartitionPositionBounds> |
getPositionsForRanges(java.util.Collection<Range<Token>> ranges)
Determine the minimal set of sections that can be extracted from this SSTable to cover the given ranges.
|
RestorableMeter |
getReadMeter() |
long |
getRecentBloomFilterFalsePositiveCount() |
long |
getRecentBloomFilterTrueNegativeCount() |
long |
getRecentBloomFilterTruePositiveCount() |
long |
getRepairedAt() |
abstract ISSTableScanner |
getScanner()
Direct I/O SSTableScanner over the entirety of the sstable..
|
abstract ISSTableScanner |
getScanner(java.util.Collection<Range<Token>> ranges)
Direct I/O SSTableScanner over a defined collection of ranges of tokens.
|
abstract ISSTableScanner |
getScanner(ColumnFilter columns,
DataRange dataRange,
SSTableReadsListener listener) |
abstract ISSTableScanner |
getScanner(java.util.Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
Direct I/O SSTableScanner over an iterator of bounds.
|
ISSTableScanner |
getScanner(Range<Token> range)
Direct I/O SSTableScanner over a defined range of tokens.
|
int |
getSSTableLevel() |
StatsMetadata |
getSSTableMetadata() |
static long |
getTotalBytes(java.lang.Iterable<SSTableReader> sstables) |
long |
getTotalColumnsSet() |
long |
getTotalRows() |
static long |
getTotalUncompressedBytes(java.lang.Iterable<SSTableReader> sstables) |
int |
hashCode() |
void |
incrementReadCount()
Increment the total read count and read rate for this SSTable.
|
static SSTableReader |
internalOpen(Descriptor desc,
java.util.Set<Component> components,
TableMetadataRef metadata,
FileHandle ifile,
FileHandle dfile,
IndexSummary summary,
IFilter bf,
long maxDataAge,
StatsMetadata sstableMetadata,
SSTableReader.OpenReason openReason,
SerializationHeader header)
Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
|
boolean |
intersects(java.util.Collection<Range<Token>> ranges) |
boolean |
isKeyCacheEnabled() |
boolean |
isMarkedCompacted() |
boolean |
isMarkedSuspect() |
boolean |
isPendingRepair() |
boolean |
isRepaired() |
boolean |
isReplaced() |
boolean |
isTransient() |
abstract UnfilteredRowIterator |
iterator(DecoratedKey key,
Slices slices,
ColumnFilter selectedColumns,
boolean reversed,
SSTableReadsListener listener) |
abstract UnfilteredRowIterator |
iterator(FileDataInput file,
DecoratedKey key,
RowIndexEntry indexEntry,
Slices slices,
ColumnFilter selectedColumns,
boolean reversed) |
DecoratedKey |
keyAt(long indexPosition) |
void |
markObsolete(java.lang.Runnable tidier)
Mark the sstable as obsolete, i.e., compacted into newer sstables.
|
void |
markSuspect() |
boolean |
mayHaveTombstones()
Whether the sstable may contain tombstones or if it is guaranteed to not contain any.
|
static SSTableReader |
moveAndOpenSSTable(ColumnFamilyStore cfs,
Descriptor oldDescriptor,
Descriptor newDescriptor,
java.util.Set<Component> components,
boolean copyData)
Moves the sstable in oldDescriptor to a new place (with generation etc) in newDescriptor.
|
void |
mutateLevelAndReload(int newLevel)
Mutate sstable level with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
|
void |
mutateRepairedAndReload(long newRepairedAt,
java.util.UUID newPendingRepair,
boolean isTransient)
Mutate sstable repair metadata with a lock to avoid racing with entire-sstable-streaming and then reload sstable metadata
|
boolean |
newSince(long age)
Tests if the sstable contains data newer than the given age param (in localhost currentMilli time).
|
long |
onDiskLength() |
static SSTableReader |
open(Descriptor descriptor) |
static SSTableReader |
open(Descriptor descriptor,
java.util.Set<Component> components,
TableMetadataRef metadata) |
static SSTableReader |
open(Descriptor descriptor,
java.util.Set<Component> components,
TableMetadataRef metadata,
boolean validate,
boolean isOffline)
Open an SSTable for reading
|
static SSTableReader |
open(Descriptor desc,
TableMetadataRef metadata) |
static java.util.Collection<SSTableReader> |
openAll(java.util.Set<java.util.Map.Entry<Descriptor,java.util.Set<Component>>> entries,
TableMetadataRef metadata) |
RandomAccessReader |
openDataReader() |
RandomAccessReader |
openDataReader(com.google.common.util.concurrent.RateLimiter limiter) |
static SSTableReader |
openForBatch(Descriptor descriptor,
java.util.Set<Component> components,
TableMetadataRef metadata)
Open SSTable reader to be used in batch mode(such as sstableloader).
|
RandomAccessReader |
openIndexReader() |
static SSTableReader |
openNoValidation(Descriptor descriptor,
java.util.Set<Component> components,
ColumnFamilyStore cfs) |
static SSTableReader |
openNoValidation(Descriptor descriptor,
TableMetadataRef metadata) |
void |
overrideReadMeter(RestorableMeter readMeter) |
Ref<SSTableReader> |
ref() |
void |
releaseSummary() |
void |
reloadSSTableMetadata()
Reloads the sstable metadata from disk.
|
static void |
resetTidying() |
void |
runOnClose(java.lang.Runnable runOnClose) |
<R> R |
runWithLock(CheckedFunction<Descriptor,R,java.io.IOException> task)
Execute provided task with sstable lock to avoid racing with index summary redistribution, SEE CASSANDRA-15861.
|
static void |
saveBloomFilter(Descriptor descriptor,
IFilter filter) |
static void |
saveSummary(Descriptor descriptor,
DecoratedKey first,
DecoratedKey last,
IndexSummary summary)
Save index summary to Summary.db file.
|
Ref<SSTableReader> |
selfRef() |
void |
setCrcCheckChance(double crcCheckChance)
Set the value of CRC check chance.
|
void |
setReplaced() |
void |
setupOnline() |
static void |
shutdownBlocking(long timeout,
java.util.concurrent.TimeUnit unit) |
abstract UnfilteredRowIterator |
simpleIterator(FileDataInput file,
DecoratedKey key,
RowIndexEntry indexEntry,
boolean tombstoneOnly) |
EncodingStats |
stats() |
Ref<SSTableReader> |
tryRef() |
long |
uncompressedLength() |
void |
unmarkSuspect() |
static void |
verifyCompressionInfoExistenceIfApplicable(Descriptor descriptor,
java.util.Set<Component> actualComponents)
Best-effort checking to verify the expected compression info component exists, according to the TOC file.
|
addComponents, appendTOC, bytesOnDisk, componentsFor, decorateKey, delete, discoverComponentsFor, estimateRowsFromIndex, getAllFilePaths, getBounds, getColumnFamilyName, getIndexFilename, getKeyspaceName, getMinimalKey, getPartitioner, metadata, readTOC, readTOC, toString, tryComponentFromFilename, tryDescriptorFromFilename, validateRepairedMetadata
public static final java.util.Comparator<SSTableReader> maxTimestampDescending
public static final java.util.Comparator<SSTableReader> maxTimestampAscending
public static final java.util.Comparator<SSTableReader> sstableComparator
public static final java.util.Comparator<SSTableReader> generationReverseComparator
public static final com.google.common.collect.Ordering<SSTableReader> sstableOrdering
public static final java.util.Comparator<SSTableReader> sizeComparator
public final long maxDataAge
public final SSTableReader.OpenReason openReason
public final SSTableReader.UniqueIdentifier instanceId
protected final FileHandle ifile
protected final FileHandle dfile
protected final IFilter bf
public final IndexSummary indexSummary
protected final RowIndexEntry.IndexSerializer<?> rowIndexEntrySerializer
protected InstrumentingCache<KeyCacheKey,RowIndexEntry> keyCache
protected final BloomFilterTracker bloomFilterTracker
protected final java.util.concurrent.atomic.AtomicBoolean isSuspect
protected volatile StatsMetadata sstableMetadata
public final SerializationHeader header
protected final java.util.concurrent.atomic.AtomicLong keyCacheHit
protected final java.util.concurrent.atomic.AtomicLong keyCacheRequest
protected SSTableReader(SSTableReaderBuilder builder)
protected SSTableReader(Descriptor desc, java.util.Set<Component> components, TableMetadataRef metadata, long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header, IndexSummary summary, FileHandle dfile, FileHandle ifile, IFilter bf)
public static long getApproximateKeyCount(java.lang.Iterable<SSTableReader> sstables)
sstables
- SSTables to calculate key countpublic static double estimateCompactionGain(java.util.Set<SSTableReader> overlapping)
public static SSTableReader open(Descriptor descriptor)
public static SSTableReader open(Descriptor desc, TableMetadataRef metadata)
public static SSTableReader open(Descriptor descriptor, java.util.Set<Component> components, TableMetadataRef metadata)
public static SSTableReader openNoValidation(Descriptor descriptor, java.util.Set<Component> components, ColumnFamilyStore cfs)
public static SSTableReader openNoValidation(Descriptor descriptor, TableMetadataRef metadata)
public static SSTableReader openForBatch(Descriptor descriptor, java.util.Set<Component> components, TableMetadataRef metadata)
descriptor
- components
- metadata
- java.io.IOException
public static SSTableReader open(Descriptor descriptor, java.util.Set<Component> components, TableMetadataRef metadata, boolean validate, boolean isOffline)
descriptor
- SSTable to opencomponents
- Components included with this SSTablemetadata
- for this SSTables CFvalidate
- Check SSTable for corruption (limited)isOffline
- Whether we are opening this SSTable "offline", for example from an external tool or not for inclusion in queries (validations)
This stops regenerating BF + Summaries and also disables tracking of hotness for the SSTable.SSTableReader
java.io.IOException
public static java.util.Collection<SSTableReader> openAll(java.util.Set<java.util.Map.Entry<Descriptor,java.util.Set<Component>>> entries, TableMetadataRef metadata)
public static SSTableReader internalOpen(Descriptor desc, java.util.Set<Component> components, TableMetadataRef metadata, FileHandle ifile, FileHandle dfile, IndexSummary summary, IFilter bf, long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
public static void verifyCompressionInfoExistenceIfApplicable(Descriptor descriptor, java.util.Set<Component> actualComponents) throws CorruptSSTableException, FSReadError
descriptor
- actualComponents,
- actual components listed from the file system.CorruptSSTableException,
- if TOC expects compression info but not found from disk.FSReadError,
- if unable to read from TOC file.CorruptSSTableException
FSReadError
public static long getTotalBytes(java.lang.Iterable<SSTableReader> sstables)
public static long getTotalUncompressedBytes(java.lang.Iterable<SSTableReader> sstables)
public boolean equals(java.lang.Object that)
equals
in class java.lang.Object
public int hashCode()
hashCode
in class java.lang.Object
public java.lang.String getFilename()
getFilename
in class SSTable
public void setupOnline()
public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last, IndexSummary summary)
public static void saveBloomFilter(Descriptor descriptor, IFilter filter)
public <R> R runWithLock(CheckedFunction<Descriptor,R,java.io.IOException> task) throws java.io.IOException
task
- to be guarded by sstable lockjava.io.IOException
public void setReplaced()
public boolean isReplaced()
public void runOnClose(java.lang.Runnable runOnClose)
public SSTableReader cloneAndReplace(IFilter newBloomFilter)
newBloomFilter
- for the replacementpublic SSTableReader cloneWithRestoredStart(DecoratedKey restoredStart)
public SSTableReader cloneWithNewStart(DecoratedKey newStart, java.lang.Runnable runOnClose)
public SSTableReader cloneWithNewSummarySamplingLevel(ColumnFamilyStore parent, int samplingLevel) throws java.io.IOException
samplingLevel
- the desired sampling level for the index summary on the new SSTableReaderjava.io.IOException
public RestorableMeter getReadMeter()
public int getIndexSummarySamplingLevel()
public long getIndexSummaryOffHeapSize()
public int getMinIndexInterval()
public double getEffectiveIndexInterval()
public void releaseSummary()
public long getIndexScanPosition(PartitionPosition key)
value >= 0
public static long getIndexScanPositionFromBinarySearchResult(int binarySearchResult, IndexSummary referencedIndexSummary)
public static int getIndexSummaryIndexFromBinarySearchResult(int binarySearchResult)
public CompressionMetadata getCompressionMetadata()
java.lang.IllegalStateException
- if the sstable is not compressedpublic long getCompressionMetadataOffHeapSize()
public IFilter getBloomFilter()
public long getBloomFilterSerializedSize()
public long getBloomFilterOffHeapSize()
public long estimatedKeys()
public long estimatedKeysForRanges(java.util.Collection<Range<Token>> ranges)
ranges
- public int getIndexSummarySize()
public int getMaxIndexSummarySize()
public byte[] getIndexSummaryKey(int index)
public java.lang.Iterable<DecoratedKey> getKeySamples(Range<Token> range)
public java.util.List<SSTableReader.PartitionPositionBounds> getPositionsForRanges(java.util.Collection<Range<Token>> ranges)
public KeyCacheKey getCacheKey(DecoratedKey key)
public void cacheKey(DecoratedKey key, RowIndexEntry info)
public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
protected RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
public boolean isKeyCacheEnabled()
public final RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op)
key
- The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
allow key selection by token bounds but only if op != * EQop
- The Operator defining matching keys: the nearest key to the target matching the operator wins.public final RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op, SSTableReadsListener listener)
key
- The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
allow key selection by token bounds but only if op != * EQop
- The Operator defining matching keys: the nearest key to the target matching the operator wins.listener
- the SSTableReaderListener
that must handle the notifications.public final RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op, boolean updateCacheAndStats)
protected abstract RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast, SSTableReadsListener listener)
key
- The key to apply as the rhs to the given Operator. A 'fake' key is allowed to
allow key selection by token bounds but only if op != * EQop
- The Operator defining matching keys: the nearest key to the target matching the operator wins.updateCacheAndStats
- true if updating stats and cachelistener
- a listener used to handle internal eventspublic abstract UnfilteredRowIterator iterator(DecoratedKey key, Slices slices, ColumnFilter selectedColumns, boolean reversed, SSTableReadsListener listener)
public abstract UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed)
public abstract UnfilteredRowIterator simpleIterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, boolean tombstoneOnly)
public DecoratedKey firstKeyBeyond(PartitionPosition token)
public long uncompressedLength()
public long onDiskLength()
public double getCrcCheckChance()
public void setCrcCheckChance(double crcCheckChance)
crcCheckChance
- public void markObsolete(java.lang.Runnable tidier)
public boolean isMarkedCompacted()
public void markSuspect()
public void unmarkSuspect()
public boolean isMarkedSuspect()
public ISSTableScanner getScanner(Range<Token> range)
range
- the range of keys to coverpublic abstract ISSTableScanner getScanner()
public abstract ISSTableScanner getScanner(java.util.Collection<Range<Token>> ranges)
ranges
- the range of keys to coverpublic abstract ISSTableScanner getScanner(java.util.Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
rangeIterator
- the keys to coverpublic abstract ISSTableScanner getScanner(ColumnFilter columns, DataRange dataRange, SSTableReadsListener listener)
columns
- the columns to return.dataRange
- filter to use when reading the columnslistener
- a listener used to handle internal read eventspublic FileDataInput getFileDataInput(long position)
public boolean newSince(long age)
age
- The age to compare the maxDataAre of this sstable. Measured in millisec since epoc on this hostpublic void createLinks(java.lang.String snapshotDirectoryPath)
public void createLinks(java.lang.String snapshotDirectoryPath, com.google.common.util.concurrent.RateLimiter rateLimiter)
public static void createLinks(Descriptor descriptor, java.util.Set<Component> components, java.lang.String snapshotDirectoryPath)
public static void createLinks(Descriptor descriptor, java.util.Set<Component> components, java.lang.String snapshotDirectoryPath, com.google.common.util.concurrent.RateLimiter limiter)
public boolean isRepaired()
public DecoratedKey keyAt(long indexPosition) throws java.io.IOException
java.io.IOException
public boolean isPendingRepair()
public java.util.UUID getPendingRepair()
public long getRepairedAt()
public boolean isTransient()
public long getBloomFilterFalsePositiveCount()
public long getRecentBloomFilterFalsePositiveCount()
public long getBloomFilterTruePositiveCount()
public long getRecentBloomFilterTruePositiveCount()
public long getBloomFilterTrueNegativeCount()
public long getRecentBloomFilterTrueNegativeCount()
public InstrumentingCache<KeyCacheKey,RowIndexEntry> getKeyCache()
public EstimatedHistogram getEstimatedPartitionSize()
public EstimatedHistogram getEstimatedCellPerPartitionCount()
public double getEstimatedDroppableTombstoneRatio(int gcBefore)
public double getDroppableTombstonesBefore(int gcBefore)
public double getCompressionRatio()
public long getMinTimestamp()
public long getMaxTimestamp()
public int getMinLocalDeletionTime()
public int getMaxLocalDeletionTime()
public boolean mayHaveTombstones()
Note that having that method return false
guarantees the sstable has no tombstones whatsoever (so no
cell tombstone, no range tombstone maker and no expiring columns), but having it return true
doesn't
guarantee it contains any as it may simply have non-expired cells.
public int getMinTTL()
public int getMaxTTL()
public long getTotalColumnsSet()
public long getTotalRows()
public int getAvgColumnSetPerRow()
public int getSSTableLevel()
public void mutateLevelAndReload(int newLevel) throws java.io.IOException
java.io.IOException
public void mutateRepairedAndReload(long newRepairedAt, java.util.UUID newPendingRepair, boolean isTransient) throws java.io.IOException
java.io.IOException
public void reloadSSTableMetadata() throws java.io.IOException
java.io.IOException
public StatsMetadata getSSTableMetadata()
public RandomAccessReader openDataReader(com.google.common.util.concurrent.RateLimiter limiter)
public RandomAccessReader openDataReader()
public RandomAccessReader openIndexReader()
public ChannelProxy getDataChannel()
public ChannelProxy getIndexChannel()
public FileHandle getIndexFile()
public long getCreationTimeFor(Component component)
component
- component to get timestamp.public long getKeyCacheHit()
public long getKeyCacheRequest()
public void incrementReadCount()
public EncodingStats stats()
public Ref<SSTableReader> tryRef()
tryRef
in interface RefCounted<SSTableReader>
public Ref<SSTableReader> selfRef()
selfRef
in interface SelfRefCounted<SSTableReader>
public Ref<SSTableReader> ref()
ref
in interface RefCounted<SSTableReader>
public void overrideReadMeter(RestorableMeter readMeter)
public void addTo(Ref.IdentityCollection identities)
public static void resetTidying()
public static SSTableReader moveAndOpenSSTable(ColumnFamilyStore cfs, Descriptor oldDescriptor, Descriptor newDescriptor, java.util.Set<Component> components, boolean copyData)
public static void shutdownBlocking(long timeout, java.util.concurrent.TimeUnit unit) throws java.lang.InterruptedException, java.util.concurrent.TimeoutException
java.lang.InterruptedException
java.util.concurrent.TimeoutException
Copyright © 2009-2022 The Apache Software Foundation