public LeveledScanner(Collection<SSTableReader> sstables, Range<Token> range) { this.range = range; // add only sstables that intersect our range, and estimate how much data that involves this.sstables = new ArrayList<SSTableReader>(sstables.size()); long length = 0; for (SSTableReader sstable : sstables) { this.sstables.add(sstable); long estimatedKeys = sstable.estimatedKeys(); double estKeysInRangeRatio = 1.0; if (estimatedKeys > 0 && range != null) estKeysInRangeRatio = ((double) sstable.estimatedKeysForRanges(Collections.singleton(range))) / estimatedKeys; length += sstable.uncompressedLength() * estKeysInRangeRatio; } totalLength = length; Collections.sort(this.sstables, SSTableReader.sstableComparator); sstableIterator = this.sstables.iterator(); assert sstableIterator.hasNext(); // caller should check intersecting first currentScanner = sstableIterator.next().getScanner(range, CompactionManager.instance.getRateLimiter()); }
public static SSTableReader open(Descriptor descriptor, Set<Component> components, CFMetaData metadata, IPartitioner partitioner) throws IOException { return open(descriptor, components, metadata, partitioner, true, true); }
/** * Open a RowIndexedReader which already has its state initialized (by SSTableWriter). */ static SSTableReader internalOpen(Descriptor desc, Set<Component> components, CFMetaData metadata, IPartitioner partitioner, SegmentedFile ifile, SegmentedFile dfile, IndexSummary isummary, IFilter bf, long maxDataAge, StatsMetadata sstableMetadata, OpenReason openReason) { assert desc != null && partitioner != null && ifile != null && dfile != null && isummary != null && bf != null && sstableMetadata != null; return new SSTableReader(desc, components, metadata, partitioner, ifile, dfile, isummary, bf, maxDataAge, sstableMetadata, openReason); }
public SSTableReader cloneWithNewStart(DecoratedKey newStart, final Runnable runOnClose) { synchronized (tidy.global) { assert openReason != OpenReason.EARLY; // TODO: merge with caller's firstKeyBeyond() work,to save time if (newStart.compareTo(first) > 0) { final long dataStart = getPosition(newStart, Operator.EQ).position; final long indexStart = getIndexScanPosition(newStart); this.tidy.runOnClose = new DropPageCache(dfile, dataStart, ifile, indexStart, runOnClose); } return cloneAndReplace(newStart, OpenReason.MOVED_START); } }
public static SSTableReader open(Descriptor desc, CFMetaData metadata) throws IOException { IPartitioner p = desc.cfname.contains(SECONDARY_INDEX_NAME_SEPARATOR) ? new LocalPartitioner(metadata.getKeyValidator()) : StorageService.getPartitioner(); return open(desc, componentsFor(desc), metadata, p); }
SSTableReader sstable = new SSTableReader(descriptor, components, metadata, sstable.load(validationMetadata); logger.debug("INDEX LOAD TIME for {}: {} ms.", descriptor, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start)); sstable.setup(trackHotness); if (validate) sstable.validate(); if (sstable.getKeyCache() != null) logger.debug("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity()); sstable.selfRef().release(); throw t;
RowIndexEntry cachedPosition = getCachedPosition(cacheKey, updateCacheAndStats); if (cachedPosition != null) long sampledPosition = getIndexScanPositionFromBinarySearchResult(binarySearchResult, indexSummary); int sampledIndex = getIndexSummaryIndexFromBinarySearchResult(binarySearchResult); cacheKey(decoratedKey, indexEntry); markSuspect(); throw new CorruptSSTableException(e, in.getPath());
if (!reset) newReader.setupKeyCache(); invalidateKeys.addAll(cachedKeys.keySet()); for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet()) newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue()); replacement = latest.cloneWithNewStart(newStart, null); if (lowerbound.compareTo(latest.last) >= 0) replacement = latest.cloneAsShadowed(runOnClose); DecoratedKey newStart = latest.firstKeyBeyond(lowerbound); assert newStart != null; replacement = latest.cloneWithNewStart(newStart, runOnClose);
/** * I/O SSTableScanner * @return A Scanner for seeking over the rows of the SSTable. */ public ISSTableScanner getScanner() { return getScanner((RateLimiter) null); }
SSTableReader sstable = new SSTableReader(descriptor, components, metadata, ? new CompressedSegmentedFile.Builder(null) : new BufferedSegmentedFile.Builder(); if (!sstable.loadSummary(ibuilder, dbuilder)) sstable.buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL); sstable.ifile = ibuilder.complete(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)); sstable.dfile = dbuilder.complete(sstable.descriptor.filenameFor(Component.DATA)); sstable.bf = FilterFactory.AlwaysPresent; sstable.setup(false); return sstable;
for (SSTableReader sstable : sstables) int level = sstable.getSSTableLevel(); " This is not a problem per se, but may indicate an orphaned sstable due to a failed" + " compaction not cleaned up properly.", sstable.getFilename(), level); level = -1; scanners.add(sstable.getScanner(range, CompactionManager.instance.getRateLimiter()));
SSTableReader sstable = SSTableReader.open(desc); RandomAccessReader dfile = sstable.openDataReader(); try RowIndexEntry entry = sstable.getPosition(decoratedKey, SSTableReader.Operator.EQ); if (entry == null) continue;
UTF8Type.instance, UTF8Type.instance); SSTableReader reader = SSTableReader.open(desc, cfm); SSTableScanner scanner = reader.getScanner();
long oldSize = bytesOnDisk(); newSummary = buildSummaryAtLevel(samplingLevel); ? SegmentedFile.getCompressedBuilder() : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode()); saveSummary(ibuilder, dbuilder, newSummary); long newSize = bytesOnDisk(); StorageMetrics.load.inc(newSize - oldSize); parent.metric.liveDiskSpaceUsed.inc(newSize - oldSize); return cloneAndReplace(first, OpenReason.METADATA_CHANGE, newSummary);
: SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode()); boolean summaryLoaded = loadSummary(ibuilder, dbuilder); boolean builtSummary = false; if (recreateBloomFilter || !summaryLoaded) buildSummary(recreateBloomFilter, ibuilder, dbuilder, summaryLoaded, Downsampling.BASE_SAMPLING_LEVEL); builtSummary = true; if (!descriptor.version.hasSamplingLevel && !builtSummary && !validateSummarySamplingLevel() && ifile != null) ? SegmentedFile.getCompressedBuilder() : SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode()); buildSummary(false, ibuilder, dbuilder, false, Downsampling.BASE_SAMPLING_LEVEL); ifile = ibuilder.complete(descriptor.filenameFor(Component.PRIMARY_INDEX)); dfile = dbuilder.complete(descriptor.filenameFor(Component.DATA)); saveSummary(ibuilder, dbuilder); saveSummary(ibuilder, dbuilder);
SSTableReader.logOpenException(entry.getKey(), e); continue; try reader = SSTableReader.open(newDescriptor, entry.getValue(), metadata, partitioner); SSTableReader.logOpenException(entry.getKey(), e); continue;
private void addNewSSTablesSize(Iterable<SSTableReader> newSSTables) { for (SSTableReader sstable : newSSTables) { if (logger.isDebugEnabled()) logger.debug(String.format("adding %s to list of files tracked for %s.%s", sstable.descriptor, cfstore.keyspace.getName(), cfstore.name)); long size = sstable.bytesOnDisk(); StorageMetrics.load.inc(size); cfstore.metric.liveDiskSpaceUsed.inc(size); cfstore.metric.totalDiskSpaceUsed.inc(size); sstable.setupKeyCache(); } }
private long avgSize(List<SSTableReader> sstables) { long n = 0; for (SSTableReader sstable : sstables) n += sstable.bytesOnDisk(); return n / sstables.size(); } };
public SSTableReader cloneAsShadowed(final Runnable runOnClose) { synchronized (tidy.global) { assert openReason != OpenReason.EARLY; this.tidy.runOnClose = new DropPageCache(dfile, 0, ifile, 0, runOnClose); return cloneAndReplace(first, OpenReason.SHADOWED); } }