Skip to content

Commit

Permalink
Merge commit 'e2ecdf268a82fa3ac0f4c9fe77ab35bca33cc72a' into cassandra-3
Browse files Browse the repository at this point in the history
.11
  • Loading branch information
pcmanus committed Aug 17, 2020
2 parents 2beebbb + e2ecdf2 commit ecd23f1
Show file tree
Hide file tree
Showing 5 changed files with 1 addition and 42 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
Expand Up @@ -2,6 +2,7 @@
* Fix short read protection for GROUP BY queries (CASSANDRA-15459)
* Frozen RawTuple is not annotated with frozen in the toString method (CASSANDRA-15857)
Merged from 3.0:
* 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
24 changes: 0 additions & 24 deletions src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java
Expand Up @@ -919,7 +919,6 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam

/* add the SSTables on disk */
Collections.sort(view.sstables, SSTableReader.maxTimestampDescending);
boolean onlyUnrepaired = true;
// read sorted sstables
SSTableReadMetricsCollector metricsCollector = new SSTableReadMetricsCollector();
for (SSTableReader sstable : view.sstables)
Expand Down Expand Up @@ -993,9 +992,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 @@ -1014,26 +1010,6 @@ private UnfilteredRowIterator queryMemtableAndSSTablesInTimestampOrder(ColumnFam
cfs.metric.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), key.hashCode(), 1);
StorageHook.instance.reportRead(cfs.metadata.cfId, partitionKey());

// "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, columnFilter()));
StageManager.getStage(Stage.MUTATION).execute(() -> {
// 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 @@ -304,11 +304,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 @@ -104,7 +104,6 @@ public class CompactionStrategyManager implements INotificationConsumer
we will use the new compaction parameters.
**/
private volatile CompactionParams schemaCompactionParams;
private boolean shouldDefragment;
private boolean supportsEarlyOpen;
private int fanout;

Expand Down Expand Up @@ -216,7 +215,6 @@ private void startup()
}
repaired.forEach(AbstractCompactionStrategy::startup);
unrepaired.forEach(AbstractCompactionStrategy::startup);
shouldDefragment = repaired.get(0).shouldDefragment();
supportsEarlyOpen = repaired.get(0).supportsEarlyOpen();
fanout = (repaired.get(0) instanceof LeveledCompactionStrategy) ? ((LeveledCompactionStrategy) repaired.get(0)).getLevelFanoutSize() : LeveledCompactionStrategy.DEFAULT_LEVEL_FANOUT_SIZE;
}
Expand Down Expand Up @@ -472,11 +470,6 @@ else if (i < a.length)
return res;
}

public boolean shouldDefragment()
{
return shouldDefragment;
}

public Directories getDirectories()
{
maybeReloadDiskBoundaries();
Expand Down
Expand Up @@ -322,12 +322,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 ecd23f1

Please sign in to comment.