Skip to content

Commit

Permalink
Remove broken "defragment-on-read" optimization
Browse files Browse the repository at this point in the history
The read path for names queries has had a "defragment-on-read"
optimization for a while whereby if too many sstables are hit by the
read, the result is written back into memtable, in the hope that later
reads will only read that newly written data in a single sstable (or at
least fewer).

The principle of that optimisation does not work however as data is
written back with the same timestamp as it originally has and that means
future reads cannot know to skip older sstables (at least with the
metadata we currently store).

As such, this optimisation never saved anything and in fact added load.

The patch removes that broken code.

Patch by Sylvain Lebresne, reviewed by Aleksey Yeschenko for
CASSANDRA-15432
  • Loading branch information
pcmanus committed Aug 17, 2020
1 parent ffc8e40 commit e2ecdf2
Show file tree
Hide file tree
Showing 5 changed files with 1 addition and 45 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
3.0.22:
* Remove broken 'defrag-on-read' optimization (CASSANDRA-15432)
* Check for endpoint collision with hibernating nodes (CASSANDRA-14599)
* Operational improvements and hardening for replica filtering protection (CASSANDRA-15907)
* stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up (CASSANDRA-15191)
Expand Down
28 changes: 0 additions & 28 deletions src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
Expand Up @@ -883,7 +883,6 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam

/* add the SSTables on disk */
Collections.sort(view.sstables, SSTableReader.maxTimestampComparator);
boolean onlyUnrepaired = true;
// read sorted sstables
SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector();
for (SSTableReader sstable : view.sstables)
Expand Down Expand Up @@ -952,9 +951,6 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
if (iter.isEmpty())
continue;

if (sstable.isRepaired())
onlyUnrepaired = false;

result = add(
RTBoundValidator.validate(isForThrift() ? ThriftResultsMerger.maybeWrap(iter, nowInSec()) : iter, RTBoundValidator.Stage.SSTABLE, false),
result,
Expand All @@ -972,30 +968,6 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
DecoratedKey key = result.partitionKey();
cfs.metric.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), key.hashCode(), 1);

// "hoist up" the requested data into a more recent sstable
if (metricsCollector.getMergedSSTables() > cfs.getMinimumCompactionThreshold()
&& onlyUnrepaired
&& !cfs.isAutoCompactionDisabled()
&& cfs.getCompactionStrategyManager().shouldDefragment())
{
// !!WARNING!! if we stop copying our data to a heap-managed object,
// we will need to track the lifetime of this mutation as well
Tracing.trace("Defragmenting requested data");

try (UnfilteredRowIterator iter = result.unfilteredIterator(columnFilter(), Slices.ALL, false))
{
final Mutation mutation = new Mutation(PartitionUpdate.fromIterator(iter));
StageManager.getStage(Stage.MUTATION).execute(new Runnable()
{
public void run()
{
// skipping commitlog and index updates is fine since we're just de-fragmenting existing data
Keyspace.open(mutation.getKeyspaceName()).apply(mutation, false, false);
}
});
}
}

return result.unfilteredIterator(columnFilter(), Slices.ALL, clusteringIndexFilter().isReversed());
}

Expand Down
Expand Up @@ -299,11 +299,6 @@ public ScannerList getScanners(Collection<SSTableReader> sstables, Collection<Ra
return new ScannerList(scanners);
}

public boolean shouldDefragment()
{
return false;
}

public String getName()
{
return getClass().getSimpleName();
Expand Down
Expand Up @@ -235,12 +235,6 @@ else if (i < a.length)
return res;
}

public boolean shouldDefragment()
{
assert repaired.getClass().equals(unrepaired.getClass());
return repaired.shouldDefragment();
}

public Directories getDirectories()
{
assert repaired.getClass().equals(unrepaired.getClass());
Expand Down
Expand Up @@ -309,12 +309,6 @@ public static Map<String, String> validateOptions(Map<String, String> options) t
return uncheckedOptions;
}

@Override
public boolean shouldDefragment()
{
return true;
}

@Override
public synchronized void addSSTable(SSTableReader added)
{
Expand Down

0 comments on commit e2ecdf2

Please sign in to comment.