[
https://issues.apache.org/jira/browse/CASSANDRA-20159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yuqi Yan updated CASSANDRA-20159:
---------------------------------
Description:
We observed slow memtable flushing hence caused write latency spikes when the
cluster has large number of SSTables (~15K). I looked into the CPU profiling,
seems that most of the CPU was busy doing updates on the View from compaction.
!image-2024-12-20-06-14-54-493.png|width=570,height=476!
And taking a closer look - these checkpoint calls are mostly from
maybeReopenEarly. I don't fully understand how this early open mechanism works,
but according to some recent investigation, the checkpoint calls can become
expensive (observed in CASSANDRA-19596, CASSANDRA-20158)
replaceFlushed update also requires an entire SSTableIntervalTree rebuild,
which can take significantly longer time when number of SSTable grows
{code:java}
static Function<View, View> replaceFlushed(final Memtable memtable, final
Iterable<SSTableReader> flushed)
{
return new Function<View, View>()
{
public View apply(View view)
{
List<Memtable> flushingMemtables =
copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
assert flushingMemtables.size() == view.flushingMemtables.size() -
1;
if (flushed == null || Iterables.isEmpty(flushed))
return new View(view.liveMemtables, flushingMemtables,
view.sstablesMap,
view.compactingMap, view.intervalTree);
Map<SSTableReader, SSTableReader> sstableMap =
replace(view.sstablesMap, emptySet(), flushed);
return new View(view.liveMemtables, flushingMemtables, sstableMap,
view.compactingMap,
SSTableIntervalTree.build(sstableMap.keySet()));
}
};
} {code}
When a node is busy with compaction, the {{replaceFlushed}} update can easily
encounter contention. Assuming 1 thread doing memtable flush and 6 concurrent
compactors running full speed, these 7 threads will consume similar time (say T
ms) to generate the new View. There is only 1/7 chance that the flush can
succeed, which means the expectation finish time will be 7 * T ms...
What's worse, replaceFlushed is not only competing the View updates with
checkpoint calls from compaction. In some of my testing, I noticed that there
is a surge in unmarkCompaction calls every 1 hour (from default
index_summary_resize_interval = 60m) and this last for 2 minutes or longer.
During this 2 minutes windows, replaceFlushed finishTime is significantly
longer and we see pending mutationstage tasks starting to pile up (all of them
are waiting for memtable to be flushed) and hence writes started to timeout.
!image-2024-12-20-06-39-04-194.png|width=1259,height=292!
These unmarkCompacting calls was from
IndexSummaryRedistribution.redistributeSummaries(). If I understand it
correctly, what happens here is that
# IndexSummaryManager marks all SSTables as compacting, which is done in
one-go per cfs by iterating all cfs and add all SSTables to compactingMap
# in adjustSamplingLevels(), it calculates the sampling level for each sstable
(probably also did some updates on SSTableReader?)
# unmark compacting for the SSTables which don't need downsample
Step 3 is done *one by one* hence causing trouble here
{code:java}
if (remainingSpace > 0)
{
Pair<List<SSTableReader>, List<ResampleEntry>> result =
distributeRemainingSpace(toDownsample, remainingSpace);
toDownsample = result.right;
newSSTables.addAll(result.left);
for (SSTableReader sstable : result.left)
transactions.get(sstable.metadata().id).cancel(sstable);
} {code}
Here I have 2 proposals in improving this:
# Fix this redistributeSummaries() to group and unmark compacting for these
SSTables in one go
# Make replaceFlushed faster - by supporting addSSTables in IntervalTree. For
replaceFlushed calls we don't remove anything from the existing IntervalTree.
That being said, insert a new interval into the IntervalTree can also be fast
(at least better than rebuild the entire tree)
** The only concern here is that addSSTables might create imbalanced tree, but
we rebuild the tree very, very frequently - I think this should not be a huge
concern
Let me know what you think about this. I'm not very sure why this cancel was
done one by one, is it possible to do it in batch?
was:
We observed slow memtable flushing hence caused write latency spikes when the
cluster has large number of SSTables (~15K). I looked into the CPU profiling,
seems that most of the CPU was busy doing updates on the View from compaction.
!image-2024-12-20-06-14-54-493.png|width=570,height=476!
And taking a closer look - these checkpoint calls are mostly from
maybeReopenEarly. I don't fully understand how this early open mechanism works,
but according to some recent investigation, the checkpoint calls can become
expensive (observed in CASSANDRA-19596, CASSANDRA-20158)
replaceFlushed update also requires an entire SSTableIntervalTree rebuild,
which can take significantly longer time when number of SSTable grows
{code:java}
static Function<View, View> replaceFlushed(final Memtable memtable, final
Iterable<SSTableReader> flushed)
{
return new Function<View, View>()
{
public View apply(View view)
{
List<Memtable> flushingMemtables =
copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
assert flushingMemtables.size() == view.flushingMemtables.size() -
1;
if (flushed == null || Iterables.isEmpty(flushed))
return new View(view.liveMemtables, flushingMemtables,
view.sstablesMap,
view.compactingMap, view.intervalTree);
Map<SSTableReader, SSTableReader> sstableMap =
replace(view.sstablesMap, emptySet(), flushed);
return new View(view.liveMemtables, flushingMemtables, sstableMap,
view.compactingMap,
SSTableIntervalTree.build(sstableMap.keySet()));
}
};
} {code}
When a node is busy with compaction, the {{replaceFlushed}} update can easily
encounter contention. Assuming 1 thread doing memtable flush and 6 concurrent
compactors running full speed, these 7 threads will consume similar time (say T
ms) to generate the new View. There is only 1/7 chance that the flush can
succeed, which means the expectation finish time will be 7 * T ms...
What's worse, replaceFlushed is not only competing the View updates with
checkpoint calls from compaction. In some of my testing, I noticed that there
is a surge in unmarkCompaction calls every 1 hour (from default
index_summary_resize_interval = 60m) and this last for 2 minutes or longer.
During this 2 minutes windows, replaceFlushed finishTime is significantly
longer and we see pending mutationstage tasks starting to pile up (all of them
are waiting for memtable to be flushed) and hence writes started to timeout.
!image-2024-12-20-06-39-04-194.png|width=1259,height=292!
These unmarkCompacting calls was from
IndexSummaryRedistribution.redistributeSummaries(). If I understand it
correctly, what happens here is that
# IndexSummaryManager marks all SSTables as compacting, which is done in
one-go per cfs by iterating all cfs and add all SSTables to compactingMap
# in adjustSamplingLevels(), it calculates the sampling level for each sstable
(probably also did some updates on SSTableReader?)
# unmark compacting for the SSTables which don't need downsample
Step 3 is done *one by one* hence causing trouble here
{code:java}
if (remainingSpace > 0)
{
Pair<List<SSTableReader>, List<ResampleEntry>> result =
distributeRemainingSpace(toDownsample, remainingSpace);
toDownsample = result.right;
newSSTables.addAll(result.left);
for (SSTableReader sstable : result.left)
transactions.get(sstable.metadata().id).cancel(sstable);
} {code}
Here I have 2 proposals in improving this:
# Fix this redistributeSummaries() to group and unmark compacting for these
SSTables in one go
# Make replaceFlushed faster - by supporting addSSTables in IntervalTree. For
replaceFlushed calls we don't remove anything from the existing IntervalTree.
That being said, insert a new interval into the IntervalTree can also be fast
(at least better than rebuild the entire tree)
** The only concern here is that addSSTables might create imbalanced tree, but
we rebuild the tree very, very frequently - I think this should not be a huge
concern
Let me know what you think about this. I have a patch for Proposal 2 and am
still working on Proposal 1.
> memtable flush stuck for minutes on slow replaceFlushed view update
> -------------------------------------------------------------------
>
> Key: CASSANDRA-20159
> URL: https://issues.apache.org/jira/browse/CASSANDRA-20159
> Project: Apache Cassandra
> Issue Type: Improvement
> Reporter: Yuqi Yan
> Assignee: Yuqi Yan
> Priority: Normal
> Attachments: image-2024-12-20-06-14-54-493.png,
> image-2024-12-20-06-39-04-194.png
>
>
> We observed slow memtable flushing hence caused write latency spikes when the
> cluster has large number of SSTables (~15K). I looked into the CPU profiling,
> seems that most of the CPU was busy doing updates on the View from compaction.
> !image-2024-12-20-06-14-54-493.png|width=570,height=476!
> And taking a closer look - these checkpoint calls are mostly from
> maybeReopenEarly. I don't fully understand how this early open mechanism
> works, but according to some recent investigation, the checkpoint calls can
> become expensive (observed in CASSANDRA-19596, CASSANDRA-20158)
> replaceFlushed update also requires an entire SSTableIntervalTree rebuild,
> which can take significantly longer time when number of SSTable grows
>
> {code:java}
> static Function<View, View> replaceFlushed(final Memtable memtable, final
> Iterable<SSTableReader> flushed)
> {
> return new Function<View, View>()
> {
> public View apply(View view)
> {
> List<Memtable> flushingMemtables =
> copyOf(filter(view.flushingMemtables, not(equalTo(memtable))));
> assert flushingMemtables.size() == view.flushingMemtables.size()
> - 1;
> if (flushed == null || Iterables.isEmpty(flushed))
> return new View(view.liveMemtables, flushingMemtables,
> view.sstablesMap,
> view.compactingMap, view.intervalTree);
> Map<SSTableReader, SSTableReader> sstableMap =
> replace(view.sstablesMap, emptySet(), flushed);
> return new View(view.liveMemtables, flushingMemtables,
> sstableMap, view.compactingMap,
> SSTableIntervalTree.build(sstableMap.keySet()));
> }
> };
> } {code}
> When a node is busy with compaction, the {{replaceFlushed}} update can easily
> encounter contention. Assuming 1 thread doing memtable flush and 6 concurrent
> compactors running full speed, these 7 threads will consume similar time (say
> T ms) to generate the new View. There is only 1/7 chance that the flush can
> succeed, which means the expectation finish time will be 7 * T ms...
>
> What's worse, replaceFlushed is not only competing the View updates with
> checkpoint calls from compaction. In some of my testing, I noticed that there
> is a surge in unmarkCompaction calls every 1 hour (from default
> index_summary_resize_interval = 60m) and this last for 2 minutes or longer.
> During this 2 minutes windows, replaceFlushed finishTime is significantly
> longer and we see pending mutationstage tasks starting to pile up (all of
> them are waiting for memtable to be flushed) and hence writes started to
> timeout.
> !image-2024-12-20-06-39-04-194.png|width=1259,height=292!
> These unmarkCompacting calls was from
> IndexSummaryRedistribution.redistributeSummaries(). If I understand it
> correctly, what happens here is that
> # IndexSummaryManager marks all SSTables as compacting, which is done in
> one-go per cfs by iterating all cfs and add all SSTables to compactingMap
> # in adjustSamplingLevels(), it calculates the sampling level for each
> sstable (probably also did some updates on SSTableReader?)
> # unmark compacting for the SSTables which don't need downsample
> Step 3 is done *one by one* hence causing trouble here
> {code:java}
> if (remainingSpace > 0)
> {
> Pair<List<SSTableReader>, List<ResampleEntry>> result =
> distributeRemainingSpace(toDownsample, remainingSpace);
> toDownsample = result.right;
> newSSTables.addAll(result.left);
> for (SSTableReader sstable : result.left)
> transactions.get(sstable.metadata().id).cancel(sstable);
> } {code}
>
> Here I have 2 proposals in improving this:
> # Fix this redistributeSummaries() to group and unmark compacting for these
> SSTables in one go
> # Make replaceFlushed faster - by supporting addSSTables in IntervalTree.
> For replaceFlushed calls we don't remove anything from the existing
> IntervalTree. That being said, insert a new interval into the IntervalTree
> can also be fast (at least better than rebuild the entire tree)
> ** The only concern here is that addSSTables might create imbalanced tree,
> but we rebuild the tree very, very frequently - I think this should not be a
> huge concern
> Let me know what you think about this. I'm not very sure why this cancel was
> done one by one, is it possible to do it in batch?
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]