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.OpenReason |
static class |
SSTableReader.Operator
TODO: Move someplace reusable
|
static class |
SSTableReader.SizeComparator |
static class |
SSTableReader.UniqueIdentifier |
RefCounted.Tidy
Modifier and Type | Field and Description |
---|---|
protected IFilter |
bf |
protected BloomFilterTracker |
bloomFilterTracker |
protected SegmentedFile |
dfile |
SerializationHeader |
header |
protected SegmentedFile |
ifile |
protected IndexSummary |
indexSummary |
SSTableReader.UniqueIdentifier |
instanceId |
protected java.util.concurrent.atomic.AtomicBoolean |
isSuspect |
protected InstrumentingCache<KeyCacheKey,RowIndexEntry> |
keyCache |
protected java.util.concurrent.atomic.AtomicLong |
keyCacheHit |
protected java.util.concurrent.atomic.AtomicLong |
keyCacheRequest |
long |
maxDataAge
maxDataAge is a timestamp in local server time (e.g.
|
static java.util.Comparator<SSTableReader> |
maxTimestampComparator |
SSTableReader.OpenReason |
openReason |
protected RowIndexEntry.IndexSerializer |
rowIndexEntrySerializer |
static java.util.Comparator<SSTableReader> |
sstableComparator |
protected StatsMetadata |
sstableMetadata |
static com.google.common.collect.Ordering<SSTableReader> |
sstableOrdering |
components, compression, descriptor, first, last, metadata, 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,
CFMetaData metadata,
long maxDataAge,
StatsMetadata sstableMetadata,
SSTableReader.OpenReason openReason,
SerializationHeader header) |
Modifier and Type | Method and Description |
---|---|
void |
addTo(Ref.IdentityCollection identities) |
void |
cacheKey(DecoratedKey key,
RowIndexEntry info) |
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) |
void |
createLinks(java.lang.String snapshotDirectoryPath) |
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.
|
void |
forceFilterFailures()
For testing purposes only.
|
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 |
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 |
getEstimatedColumnCount() |
double |
getEstimatedDroppableTombstoneRatio(int gcBefore) |
EstimatedHistogram |
getEstimatedPartitionSize() |
FileDataInput |
getFileDataInput(long position) |
java.lang.String |
getFilename() |
ChannelProxy |
getIndexChannel() |
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() |
RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op)
Get position updating key cache and stats.
|
RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op,
boolean updateCacheAndStats) |
protected abstract RowIndexEntry |
getPosition(PartitionPosition key,
SSTableReader.Operator op,
boolean updateCacheAndStats,
boolean permitMatchPastLast) |
java.util.List<Pair<java.lang.Long,java.lang.Long>> |
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 |
getRecentBloomFilterTruePositiveCount() |
ISSTableScanner |
getScanner()
I/O SSTableScanner
|
abstract ISSTableScanner |
getScanner(java.util.Collection<Range<Token>> ranges,
com.google.common.util.concurrent.RateLimiter limiter)
Direct I/O SSTableScanner over a defined collection of ranges of tokens.
|
ISSTableScanner |
getScanner(ColumnFilter columns,
DataRange dataRange,
boolean isForThrift) |
abstract ISSTableScanner |
getScanner(ColumnFilter columns,
DataRange dataRange,
com.google.common.util.concurrent.RateLimiter limiter,
boolean isForThrift) |
abstract ISSTableScanner |
getScanner(java.util.Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
Direct I/O SSTableScanner over an iterator of bounds.
|
ISSTableScanner |
getScanner(Range<Token> range,
com.google.common.util.concurrent.RateLimiter limiter)
Direct I/O SSTableScanner over a defined range of tokens.
|
abstract ISSTableScanner |
getScanner(com.google.common.util.concurrent.RateLimiter limiter)
Direct I/O SSTableScanner over the entirety of the sstable..
|
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() |
boolean |
hasTombstones()
sstable contains no tombstones if minLocalDeletionTime == Integer.MAX_VALUE
|
void |
incrementReadCount()
Increment the total row read count and read rate for this SSTable.
|
static SSTableReader |
internalOpen(Descriptor desc,
java.util.Set<Component> components,
CFMetaData metadata,
SegmentedFile ifile,
SegmentedFile dfile,
IndexSummary isummary,
IFilter bf,
long maxDataAge,
StatsMetadata sstableMetadata,
SSTableReader.OpenReason openReason,
SerializationHeader header)
Open a RowIndexedReader which already has its state initialized (by SSTableWriter).
|
boolean |
isKeyCacheSetup() |
boolean |
isMarkedCompacted() |
boolean |
isMarkedSuspect() |
boolean |
isRepaired() |
boolean |
isReplaced() |
abstract SliceableUnfilteredRowIterator |
iterator(DecoratedKey key,
ColumnFilter selectedColumns,
boolean reversed,
boolean isForThrift) |
abstract SliceableUnfilteredRowIterator |
iterator(FileDataInput file,
DecoratedKey key,
RowIndexEntry indexEntry,
ColumnFilter selectedColumns,
boolean reversed,
boolean isForThrift) |
boolean |
loadSummary(SegmentedFile.Builder ibuilder,
SegmentedFile.Builder dbuilder)
Load index summary from Summary.db file if it exists.
|
static void |
logOpenException(Descriptor descriptor,
java.io.IOException e) |
void |
markObsolete(java.lang.Runnable tidier)
Mark the sstable as obsolete, i.e., compacted into newer sstables.
|
void |
markSuspect() |
boolean |
mayOverlapsWith(SSTableReader other)
Checks if this sstable can overlap with another one based on the min/man clustering values.
|
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 desc,
CFMetaData metadata) |
static SSTableReader |
open(Descriptor descriptor,
java.util.Set<Component> components,
CFMetaData metadata) |
static SSTableReader |
open(Descriptor descriptor,
java.util.Set<Component> components,
CFMetaData metadata,
boolean validate,
boolean trackHotness) |
static java.util.Collection<SSTableReader> |
openAll(java.util.Set<java.util.Map.Entry<Descriptor,java.util.Set<Component>>> entries,
CFMetaData metadata) |
RandomAccessReader |
openDataReader() |
RandomAccessReader |
openDataReader(com.google.common.util.concurrent.RateLimiter limiter) |
static SSTableReader |
openForBatch(Descriptor descriptor,
java.util.Set<Component> components,
CFMetaData metadata)
Open SSTable reader to be used in batch mode(such as sstableloader).
|
RandomAccessReader |
openIndexReader() |
static SSTableReader |
openNoValidation(Descriptor descriptor,
CFMetaData metadata) |
static SSTableReader |
openNoValidation(Descriptor descriptor,
java.util.Set<Component> components,
ColumnFamilyStore cfs) |
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) |
static void |
saveSummary(Descriptor descriptor,
DecoratedKey first,
DecoratedKey last,
SegmentedFile.Builder ibuilder,
SegmentedFile.Builder dbuilder,
IndexSummary summary)
Save index summary to Summary.db file.
|
void |
saveSummary(SegmentedFile.Builder ibuilder,
SegmentedFile.Builder dbuilder)
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() |
Ref<SSTableReader> |
tryRef() |
long |
uncompressedLength() |
addComponents, appendTOC, bytesOnDisk, componentsFor, decorateKey, delete, discoverComponentsFor, estimateRowsFromIndex, getAllFilePaths, getColumnFamilyName, getIndexFilename, getKeyspaceName, getMinimalKey, getPartitioner, readTOC, toString, tryComponentFromFilename
public static final java.util.Comparator<SSTableReader> maxTimestampComparator
public static final java.util.Comparator<SSTableReader> sstableComparator
public static final com.google.common.collect.Ordering<SSTableReader> sstableOrdering
public final long maxDataAge
public final SSTableReader.OpenReason openReason
public final SSTableReader.UniqueIdentifier instanceId
protected SegmentedFile ifile
protected SegmentedFile dfile
protected IndexSummary indexSummary
protected IFilter bf
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(Descriptor desc, java.util.Set<Component> components, CFMetaData metadata, long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
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) throws java.io.IOException
java.io.IOException
public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws java.io.IOException
java.io.IOException
public static SSTableReader open(Descriptor descriptor, java.util.Set<Component> components, CFMetaData metadata) throws java.io.IOException
java.io.IOException
public static SSTableReader openNoValidation(Descriptor descriptor, java.util.Set<Component> components, ColumnFamilyStore cfs) throws java.io.IOException
java.io.IOException
public static SSTableReader openNoValidation(Descriptor descriptor, CFMetaData metadata) throws java.io.IOException
java.io.IOException
public static SSTableReader openForBatch(Descriptor descriptor, java.util.Set<Component> components, CFMetaData metadata) throws java.io.IOException
descriptor
- components
- metadata
- java.io.IOException
public static SSTableReader open(Descriptor descriptor, java.util.Set<Component> components, CFMetaData metadata, boolean validate, boolean trackHotness) throws java.io.IOException
java.io.IOException
public static void logOpenException(Descriptor descriptor, java.io.IOException e)
public static java.util.Collection<SSTableReader> openAll(java.util.Set<java.util.Map.Entry<Descriptor,java.util.Set<Component>>> entries, CFMetaData metadata)
public static SSTableReader internalOpen(Descriptor desc, java.util.Set<Component> components, CFMetaData metadata, SegmentedFile ifile, SegmentedFile dfile, IndexSummary isummary, IFilter bf, long maxDataAge, StatsMetadata sstableMetadata, SSTableReader.OpenReason openReason, SerializationHeader header)
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 boolean isKeyCacheSetup()
public boolean loadSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
ibuilder
- dbuilder
- public void saveSummary(SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder)
ibuilder
- dbuilder
- public static void saveSummary(Descriptor descriptor, DecoratedKey first, DecoratedKey last, SegmentedFile.Builder ibuilder, SegmentedFile.Builder dbuilder, IndexSummary summary)
public void setReplaced()
public boolean isReplaced()
public void runOnClose(java.lang.Runnable runOnClose)
public 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)
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 void forceFilterFailures()
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<Pair<java.lang.Long,java.lang.Long>> 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 RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op)
public RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op, boolean updateCacheAndStats)
protected abstract RowIndexEntry getPosition(PartitionPosition key, SSTableReader.Operator op, boolean updateCacheAndStats, boolean permitMatchPastLast)
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 cachepublic abstract SliceableUnfilteredRowIterator iterator(DecoratedKey key, ColumnFilter selectedColumns, boolean reversed, boolean isForThrift)
public abstract SliceableUnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, ColumnFilter selectedColumns, boolean reversed, boolean isForThrift)
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 boolean isMarkedSuspect()
public ISSTableScanner getScanner()
public ISSTableScanner getScanner(ColumnFilter columns, DataRange dataRange, boolean isForThrift)
columns
- the columns to return.dataRange
- filter to use when reading the columnspublic ISSTableScanner getScanner(Range<Token> range, com.google.common.util.concurrent.RateLimiter limiter)
range
- the range of keys to coverpublic abstract ISSTableScanner getScanner(com.google.common.util.concurrent.RateLimiter limiter)
public abstract ISSTableScanner getScanner(java.util.Collection<Range<Token>> ranges, com.google.common.util.concurrent.RateLimiter limiter)
ranges
- the range of keys to coverpublic abstract ISSTableScanner getScanner(java.util.Iterator<AbstractBounds<PartitionPosition>> rangeIterator)
bounds
- the keys to coverpublic abstract ISSTableScanner getScanner(ColumnFilter columns, DataRange dataRange, com.google.common.util.concurrent.RateLimiter limiter, boolean isForThrift)
columns
- the columns to return.dataRange
- filter to use when reading the columnspublic 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 boolean isRepaired()
public long getBloomFilterFalsePositiveCount()
public long getRecentBloomFilterFalsePositiveCount()
public long getBloomFilterTruePositiveCount()
public long getRecentBloomFilterTruePositiveCount()
public InstrumentingCache<KeyCacheKey,RowIndexEntry> getKeyCache()
public EstimatedHistogram getEstimatedPartitionSize()
public EstimatedHistogram getEstimatedColumnCount()
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 hasTombstones()
public int getMinTTL()
public int getMaxTTL()
public long getTotalColumnsSet()
public long getTotalRows()
public int getAvgColumnSetPerRow()
public int getSSTableLevel()
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 long getCreationTimeFor(Component component)
component
- component to get timestamp.public long getKeyCacheHit()
public long getKeyCacheRequest()
public void incrementReadCount()
public boolean mayOverlapsWith(SSTableReader other)
this
and other
have no overlapping
data, i.e. no cells to reconcile.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()
Copyright © 2017 The Apache Software Foundation