Files
scylladb/compaction
Łukasz Paszkowski 4f5d10ccd0 compaction_manager: fix maybe_wait_for_sstable_count_reduction() hanging forever
The futurization refactoring in 9d3755f276 ("replica: Futurize
retrieval of sstable sets in compaction_group_view") changed
maybe_wait_for_sstable_count_reduction() from a single predicated
wait:
```
    co_await cstate.compaction_done.wait([..] {
        return num_runs_for_compaction() <= threshold
            || !can_perform_regular_compaction(t);
    });
```
to a while loop with a predicated wait:
```
    while (can_perform_regular_compaction(t)
           && co_await num_runs_for_compaction() > threshold) {
        co_await cstate.compaction_done.wait([this, &t] {
            return !can_perform_regular_compaction(t);
        });
    }
```

This was necessary because num_runs_for_compaction() became a
coroutine (returns future<size_t>) and can no longer be called
inside a condition_variable predicate (which must be synchronous).

However, the inner wait's predicate — !can_perform_regular_compaction(t)
— only returns true when compaction is disabled or the table is being
removed. During normal operation, every signal() from compaction_done
wakes the waiter, the predicate returns false, and the waiter
immediately goes back to sleep without ever re-checking the outer
while loop's num_runs_for_compaction() condition.

This causes memtable flushes to hang forever in
maybe_wait_for_sstable_count_reduction() whenever the sstable run
count exceeds the threshold, because completed compactions signal
compaction_done but the signal is swallowed by the predicate.

Fix by replacing the predicated wait with a bare wait(), so that
any signal (including from completed compactions) causes the outer
while loop to re-evaluate num_runs_for_compaction().

Fixes: https://scylladb.atlassian.net/browse/SCYLLADB-610

Closes scylladb/scylladb#28801

(cherry picked from commit bb57b0f3b7)
2026-02-27 01:39:05 +02:00
..