Skip to content

STAR-823: Fix deadlock in CompactionManager#230

Merged
jacek-lewandowski merged 18 commits intods-trunkfrom
STAR-823-2
Aug 10, 2021
Merged

STAR-823: Fix deadlock in CompactionManager#230
jacek-lewandowski merged 18 commits intods-trunkfrom
STAR-823-2

Conversation

@jacek-lewandowski
Copy link
Copy Markdown

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full

  • we have n background tasks there waiting for the compaction tasks
    that cannot start.

Another thing fixed in this commit (perhaps minor) is that we use
getActiveCount() on the executor to check how many tasks it is
currently running and based on that information either schedule
new tasks or not. The problem with this method is that it returns
approximate result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored and hopefully the logic is cleaner now.

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing fixed in this commit (perhaps minor) is that we use
getActiveCount() on the executor to check how many tasks it is
currently running and based on that information either schedule
new tasks or not. The problem with this method is that it returns
approximate result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored and hopefully the logic is cleaner now.
Comment thread src/java/org/apache/cassandra/db/compaction/BackgroundCompactionRunner.java Outdated
/**
* when the compaction was decided not to run because there were already running compactions for this CFS
*/
SKIPPED,
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this result makes sense. The point here is to flag that a compaction check is needed and it shouldn't matter at all whose call was the one that actually caused the check to execute.

In other words, if the threads are busy and we get a call, the returned future should be completed after:

  • a thread frees up / some compaction completes
  • a compaction check for this cfs is performed
  • any compaction it triggered is completed.

What I imagine the logic to doing this to be is one that uses just one future per cfs and does something like return compactionRequests.computeIfAbsent(cfs, CompletableFuture::new) on request, future = compactionRequests.remove(cfs) when we start the check, but only start the check if we should (i.e. cfs is not currently compacting or we have free threads), and future.complete() attached to the compaction task completion.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, that seems to be much simpler - the reason why I implemented it using a list of promises is that each returned promise can be cancelled, and if all promises for the CFS are cancelled, we do not run the compaction for that CFS.

If I implement it using a single promise per CFS, either we can ignore cancellation at all or make it explicit that cancellation aborts all the requests for the CFS.

However, we actually never cancel those futures, and the production code in fact never use the returned future.

Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Cancellation does not make sense either. We request (perhaps "flag" is actually a better name) when we know there's some reason to believe a new compaction may be necessary; I don't see a mechanism why one would want to cancel such an indication -- the compaction strategy is the component that decides if a compaction should actually be issued.

Also, doing an unnecessary extra check is not a problem, not noticing that one should be done is more serious, but still benign if we have the periodic catch-all request.

Comment thread src/java/org/apache/cassandra/db/compaction/BackgroundCompactionRunner.java Outdated

if (!maybeScheduleNextCheck())
{
futureResult.cancel(false);
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's cancel all tasks in the set on shutdown instead.

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we would have to synchronize requestCompaction and shutdown in that case, right?

cfs.name,
cfs.getCompactionStrategy().getName());

Supplier<CompletableFuture<?>> compactionTask = Optional.ofNullable(lazyRunCompactionTasks(cfs))
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: This creates some unnecessary garbage and is not that easy to see through. I would spell out

if (compactionTask == null)
    compactionTask = lazyRunUpgradeTasks(cfs);

Why is the supplier necessary?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I use the supplier because I suppose that I need to know whether there is anything to run or not before starting the tasks. If the returned supplier is null - there is nothing to run. Based on that, I can add the CFS to ongoingCompactions before actually starting the compaction tasks. I cannot add it after starting because it is a multiset and removal is tied to the completion of the task - so we would have a race.

I also didn't want to spread usages of ongoingCompactions between methods and have it modified only in a single place

{
try
{
checkExecutor.execute(this);
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The previous version also had a periodic catch-all request for every cfs. Do we still have that?

Copy link
Copy Markdown
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Without the changes suggested in #230 (comment) it is not needed because after requesting a background compaction, we are guaranteed that at least background check is run after the request is made and the request is consumed with that run (and all promises are completed)

ongoingCompactions.add(cfs);

compactionTask.get().handle((ignored, throwable) -> {
CompletableFuture<?>[] tasksResults = compactionTasks.get();
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we add some explanation that the supplier is used to guarantee ongoingCompactions is set before the tasks are started?

@blambov
Copy link
Copy Markdown

blambov commented Aug 6, 2021

I made some suggestions in #231 to try and make this logic as simple as possible. What we want is to:

  • Let CFSs be marked for background compaction.
  • When there's a free thread, check for possibilities and start compactions on the marked CFSs.
  • Clear the mark if no compaction is needed.
  • Return a future that is completed when a background check is made following a request and any initiated compaction has completed.

It changes the behaviour a little further by counting tasks as ongoing as soon as they are scheduled, which I believe makes a better sense as a indicator when we should stop scheduling anything further.

@blambov
Copy link
Copy Markdown

blambov commented Aug 6, 2021

One further question is whether we should repeatedly schedule for the same CFS; I did not make that change, but it really makes sense to do so, i.e. repeat the run loop until the set is exhausted or the threads are busy, and only remove from the set in the "NOT_NEEDED" path.

The point is that we may have multiple compaction possibilities (e.g. a few flushed sstables + a compaction getting the next level over the threshold) and the legacy compaction strategies will only issue one task at a time.

@jacek-lewandowski
Copy link
Copy Markdown
Author

One further question is whether we should repeatedly schedule for the same CFS; I did not make that change, but it really makes sense to do so, i.e. repeat the run loop until the set is exhausted or the threads are busy, and only remove from the set in the "NOT_NEEDED" path.

The point is that we may have multiple compaction possibilities (e.g. a few flushed sstables + a compaction getting the next level over the threshold) and the legacy compaction strategies will only issue one task at a time.

But that is already accomplished - when the task for CFS is completed we mark it for the compaction again - when we mark, we also make sure there is another check scheduled so we will exhaust the compaction possibilities.

For the periodic run - ColumnFamilyStore periodically trigger background compaction for each CFS (when automatic compaction is enabled)

@jacek-lewandowski
Copy link
Copy Markdown
Author

Ah, indeed, with the proposed changes it would not reschedule. What do you think about having a kind of a listener in CompactionExecutor which gets triggered when a task is finished - such a listener could trigger the background compaction check?

@blambov
Copy link
Copy Markdown

blambov commented Aug 6, 2021

But that is already accomplished - when the task for CFS is completed we mark it for the compaction again

We do ensure that we will eventually start compactions, but only serially. What the current code is not doing is to schedule multiple parallel compactions in one pass. It will trigger a check at the end, but it will be one single check while multiple compactions may have become possible. This is also the case for the old code (when the threads are busy we cancel all checks but one).

I can imagine this being a problem e.g. at a "wait for compactions" stage in fallout where no flushes happen and with some bad luck nothing is really triggering a check until a large compaction completes, while other ones could be executed in parallel.

What do you think about having a kind of a listener in CompactionExecutor which gets triggered when a task is finished - such a listener could trigger the background compaction check?

Isn't this describing what the second commit in #231 does?

@jacek-lewandowski
Copy link
Copy Markdown
Author

But that is already accomplished - when the task for CFS is completed we mark it for the compaction again

We do ensure that we will eventually start compactions, but only serially. What the current code is not doing is to schedule multiple parallel compactions in one pass. It will trigger a check at the end, but it will be one single check while multiple compactions may have become possible. This is also the case for the old code (when the threads are busy we cancel all checks but one).

It also schedules a check when we do not run the compactions because the executor pool is exhausted (https://github.com/datastax/cassandra/pull/230/files#diff-c78c3b488b692922347cf0e9cfa7feebf67db5d19223c68c51944470891581acR223) so it keeps running until it reaches the point where it gets the tasks list and there is no task to run or all the tasks failed.

What do you think about having a kind of a listener in CompactionExecutor which gets triggered when a task is finished - such a listener could trigger the background compaction check?

Isn't this describing what the second commit in #231 does?

Well, I don't see that - it seems to base on the automatic background compactions triggering by CFS, which happens rather rarely. What I meant is that the compaction executor in its afterExecute implementation checks if there are free threads and if so, it notifies the listener. The background compactions runner schedules the check when that happens. My current solution is bad because we fall in a resource intensive looping - next check is scheduled immediately when after the current one when the executor is busy.

@blambov
Copy link
Copy Markdown

blambov commented Aug 9, 2021

it keeps running until it reaches the point where it gets the tasks list and there is no task to run or all the tasks failed

Oh, I've missed this. Doesn't this mean that this thread will loop indefinitely i.e. keep a CPU thread busy, as soon as we have enough tasks for all threads?

Well, I don't see that

A thread can only free up when a task completes. Together with decreasing the counter, startTask triggers the request processing loop. We probably need a comment that this will also start pending checks for other CFSs waiting on free threads.

@blambov
Copy link
Copy Markdown

blambov commented Aug 9, 2021

For the periodic run - ColumnFamilyStore periodically trigger background compaction for each CFS (when automatic compaction is enabled)

This is one more thing we can drastically simplify -- with a common runner we can have just one periodic task.

Copy link
Copy Markdown

@blambov blambov left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the patience.

return p;
}

private FutureRequestResult requestCompactionInternal(ColumnFamilyStore cfs)
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: How about calling this markForCompactionCheck?

}
}

private boolean maybeScheduleNextCheck(Duration delay)
Copy link
Copy Markdown

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: we now always schedule with Duration.ZERO. Maybe fully revert fece018?

jacek-lewandowski added a commit that referenced this pull request Oct 17, 2022
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
jacek-lewandowski added a commit that referenced this pull request Oct 18, 2022
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
mfleming pushed a commit that referenced this pull request Jul 10, 2023
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
djatnieks pushed a commit that referenced this pull request Jul 24, 2023
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
djatnieks pushed a commit that referenced this pull request Aug 22, 2023
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)
djatnieks pushed a commit that referenced this pull request Sep 12, 2023
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed
jacek-lewandowski added a commit that referenced this pull request Jan 28, 2024
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)
djatnieks pushed a commit that referenced this pull request Mar 29, 2024
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
djatnieks pushed a commit that referenced this pull request Apr 1, 2024
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
djatnieks pushed a commit that referenced this pull request Apr 16, 2024
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
djatnieks pushed a commit that referenced this pull request Jan 30, 2025
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
djatnieks pushed a commit that referenced this pull request May 18, 2025
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
michaelsembwever pushed a commit that referenced this pull request Feb 6, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.
michaelsembwever pushed a commit that referenced this pull request Feb 10, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Feb 11, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Feb 12, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Feb 14, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Feb 16, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Feb 27, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Mar 2, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Mar 4, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Mar 25, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Mar 27, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
michaelsembwever pushed a commit that referenced this pull request Apr 14, 2026
STAR-823: Refactor background compactions

CompactionManager.BackgroundCompactionCandidate task is scheduled on
compaction executor and when it detects there are compaction tasks
to run, it starts each compaction task as a separate job on the same
compaction executor and blocks until all tasks are finished.

When the pool size for the executor is n, and there are n background
tasks submitted in parallel, and all of them find that there are some
compactions to run, they will schedule them and block until the tasks
are finished. Though, the tasks cannot start because the pool is full
- we have n background tasks there waiting for the compaction tasks
that cannot start.

Another thing (perhaps minor) is that we use getActiveCount() on
the executor to check how many tasks it is currently running,
and based on that information either schedule new tasks or not.
The problem with this method is that it returns an approximate
result and should not be used for making such decisions.

To address those problems, running of background compactions was
refactored. The whole logic for background compactions was extracted
into a distinct class BackgroundCompactionsRunner. It allows flagging
CFSs for compaction and schedules scans through the flagged CFSs
on a dedicated executor so that scans and compaction tasks are no
longer sharing the same executor.

Co-authored-by: Branimir Lambov <branimir.lambov@datastax.com>
(cherry picked from commit 96bf61c)
(cherry picked from commit 804b885)
(cherry picked from commit 2c0dcd7)
(cherry picked from commit 0d72d46)
(cherry picked from commit 4cea2b1)
(cherry picked from commit 840d804)

STAR-823 Fix CompactionManager after compactingCF was removed

(cherry picked from commit e8df796)

STAR-823 Fix rebase compile errors and adapt BackgroundCompactionRunner to use o.a.c.u.concurrent.Future that is expected in C* 5.0 instead of CompletableFuture.

STAR-823 Tell checkstyle to allow CompletableFuture in BackgroundCompactionRunner
There are still some uses of CompletableFuture in BackgroundCompactionRunner

STAR-823 Use trySuccess/tryFailure instead of setSuccess/setFailure to not throw an exception if the promise was already completed, matching what was in CC 4.0 that uses CompletableFuture.

 (Rebase of commit d051dc0)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants