-
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
Track cancellable tasks by parent ID #76186
Track cancellable tasks by parent ID #76186
Conversation
Today when cancelling a task with its descendants we perform a linear scan through all the tasks looking for the few that have the right parent ID. With potentially hundreds of thousands of tasks this takes quite some time, particularly if there are many tasks to cancel. This commit introduces a second map that tracks the tasks by their parent ID so that it's super-cheap to find the descendants that need to be cancelled. Closes elastic#75316
Pinging @elastic/es-distributed (Team:Distributed) |
My biggest concern with this right now is the |
I wrote a simple benchmark:
These times track adding and then removing all the tasks; even with 30000 parent tasks each with 1000 child tasks the mean overhead per task is ~2.2µs.
|
This reverts commit 9aedfa7.
I reverted the "optimisation" for the one-item case because the same benchmark indicates it's actually slightly slower, even in the runs when there's only one child task per parent.
|
The version in 76ed889 (== 03cf0a9) is a slight improvement at larger child counts and ~no difference otherwise.
|
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 looks great. Left a few comments around testing.
} | ||
|
||
private final Map<Long, T> byTaskId = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); | ||
private final Map<TaskId, T[]> byParentTaskId = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); |
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.
Can we extend TaskAssertions.assertAllCancellableTasksAreCancelled
and perhaps even ESIntegTestCase
to verify that the two maps are in sync after tests have completed (at least that the byParentId
map only contains items that are also in the byTaskId
map).
We could consider doing it in an assertion too but doing it concurrently while running might be difficult?
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.
Yes I don't think we can express any true invariants very easily; I added an eventually-true assertion in 9554f12.
tracker.put(task, item); | ||
state.incrementAndGet(); | ||
|
||
state.incrementAndGet(); |
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.
I am not entirely sure we would ever see state==2 in other threads, I would think it would be ok for the JVM to optimize it away.
This is possibly even true for the entire actionThread
since there is no synchronization with the watch threads here. The watch threads may only see the the final output. I would advocate a yield should help, but that is outside JMM I think.
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.
Ah, an earlier iteration had some assertions in between these calls. In theory you're right that it would be legitimate for the compiler to collapse these tests to something fairly trivial, but in practice they do cover many interleavings and the other threads do indeed observe state==2
cases. Just try adding bugs :) I'll add a yield
anyway.
watchThread = new Thread(() -> { | ||
awaitStart.run(); | ||
|
||
for (int i = 0; i < 10; i++) { |
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.
Should this not always run at least until state.get() ==4 (and then another round)?
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.
I found that this sometimes noticeably added to the test runtime, we spent ages doing this loop before actually doing the useful work. Again, in practice this covers what we care about.
task.join(); | ||
} | ||
} | ||
} |
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.
I think we should also check the more trivial case where no parent id is set.
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.
Maybe I misunderstand, I think we already do?
rarely() ? TaskId.EMPTY_TASK_ID : randomFrom(parentTaskIds),
private final Map<TaskId, T[]> byParentTaskId = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency(); | ||
|
||
/** | ||
* Add an item for the given task. Should only be called once for each task. |
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.
Also add a comment that items must be unique per task.
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.
++ although I'm not sure that's true, I believe we handle duplicates ok?
server/src/main/java/org/elasticsearch/tasks/CancellableTasksTracker.java
Show resolved
Hide resolved
*/ | ||
public class CancellableTasksTracker<T> { | ||
|
||
private final T[] empty; |
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.
Can empty
be a constant? It doesn't seems to modified anywhere and it's used in Arrays.copyOf
.
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.
Again I didn't find a way to do so, but you're welcome to show me how :)
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.
Why does this have to be a generic, T == CancellableTaskHolder
is the only use case we have isn't it?
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.
Yeah in production, I didn't want to make CancellableTaskHolder
public just for the tests and I don't think it makes a performance difference. I could be persuaded.
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.
I don't think it makes a performance difference. I could be persuaded.
Nah there isn't any difference I think lets leave it as is then IMO :)
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.
LGTM (dropping the generic may be a nit but I can see how it's more fun to have it for tests)
* Return a collection of all the tracked items. May be large. In the presence of concurrent calls to {@link #put} and {@link #remove} | ||
* it behaves similarly to {@link ConcurrentHashMap#values()}. | ||
*/ | ||
public Iterable<T> values() { |
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.
Nit: might just as well return Collection
here and simplify the copying of the result in the tests (the doc talks about a collection anyways :)).
*/ | ||
public class CancellableTasksTracker<T> { | ||
|
||
private final T[] empty; |
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.
Why does this have to be a generic, T == CancellableTaskHolder
is the only use case we have isn't it?
Today when cancelling a task with its descendants we perform a linear scan through all the tasks looking for the few that have the right parent ID. With potentially hundreds of thousands of tasks this takes quite some time, particularly if there are many tasks to cancel. This commit introduces a second map that tracks the tasks by their parent ID so that it's super-cheap to find the descendants that need to be cancelled. Closes #75316
Today when cancelling a task with its descendants we perform a linear scan through all the tasks looking for the few that have the right parent ID. With potentially hundreds of thousands of tasks this takes quite some time, particularly if there are many tasks to cancel. This commit introduces a second map that tracks the tasks by their parent ID so that it's super-cheap to find the descendants that need to be cancelled. Closes #75316
Today when cancelling a task with its descendants we perform a linear
scan through all the tasks looking for the few that have the right
parent ID. With potentially hundreds of thousands of tasks this takes
quite some time, particularly if there are many tasks to cancel.
This commit introduces a second map that tracks the tasks by their
parent ID so that it's super-cheap to find the descendants that need to
be cancelled.
Closes #75316