-
Notifications
You must be signed in to change notification settings - Fork 24.9k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Parallelize stale index deletion #100316
Parallelize stale index deletion #100316
Changes from all commits
67867ce
e13f1fb
462af51
5062861
07f759b
f607f48
6101381
2be9141
a804b22
97ed99e
b56455c
488e4bb
bdff6e2
4592ad1
3673d40
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,6 @@ | ||
pr: 100316 | ||
summary: Parallelize stale index deletion | ||
area: Snapshot/Restore | ||
type: enhancement | ||
issues: | ||
- 61513 |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,6 +17,7 @@ | |
|
||
import java.util.Queue; | ||
import java.util.concurrent.Executor; | ||
import java.util.concurrent.atomic.AtomicBoolean; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
|
||
/** | ||
|
@@ -155,4 +156,54 @@ int runningTasks() { | |
return runningTasks.get(); | ||
} | ||
|
||
/** | ||
* Run a single task on the given executor which eagerly pulls tasks from the queue and executes them. This must only be used if the | ||
* tasks in the queue are all synchronous, i.e. they release their ref before returning from {@code onResponse()}. | ||
*/ | ||
public void runSyncTasksEagerly(Executor executor) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why not have an overload version of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes that could work too. I'm going to wait and see on that idea tho, I'd rather do one or the other, and the choice depends on whether we have other users that want to do something else or whether everyone just forks another task on |
||
executor.execute(new AbstractRunnable() { | ||
@Override | ||
protected void doRun() { | ||
final AtomicBoolean isDone = new AtomicBoolean(true); | ||
final Releasable ref = () -> isDone.set(true); | ||
ActionListener<Releasable> task; | ||
while ((task = tasks.poll()) != null) { | ||
isDone.set(false); | ||
try { | ||
logger.trace("[{}] eagerly running task {}", taskRunnerName, task); | ||
task.onResponse(ref); | ||
} catch (Exception e) { | ||
logger.error(Strings.format("[%s] task %s failed", taskRunnerName, task), e); | ||
assert false : e; | ||
task.onFailure(e); | ||
return; | ||
} | ||
if (isDone.get() == false) { | ||
logger.error( | ||
"runSyncTasksEagerly() was called on a queue [{}] containing an async task: [{}]", | ||
taskRunnerName, | ||
task | ||
); | ||
assert false; | ||
return; | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public void onFailure(Exception e) { | ||
logger.error("unexpected failure in runSyncTasksEagerly", e); | ||
assert false : e; | ||
} | ||
|
||
@Override | ||
public void onRejection(Exception e) { | ||
if (e instanceof EsRejectedExecutionException) { | ||
logger.debug("runSyncTasksEagerly was rejected", e); | ||
} else { | ||
onFailure(e); | ||
} | ||
} | ||
}); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is a cool test and I learnt a few tips and tricks from it. But it does not test the new parallelization change. Do we care?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This test is largely from the original contributor, but I think it's a reasonable test to write. I've played around with a few ideas for testing the new threading more precisely but it seems pretty tricky, and kinda doesn't matter so much as long as we do actually do the work somehow. I think I had an idea for a test tho.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok my idea seems to work, see
testCleanupStaleBlobsConcurrency
added in b56455c (sorry for the force-push)