perf: improve RemoteTaskRunner task assignment loop performance#12096
perf: improve RemoteTaskRunner task assignment loop performance#12096samarthjain merged 2 commits intoapache:masterfrom
Conversation
|
Good catch. Overall LGTM. Leave a comment to you to see if we can improve more. |
| return parser.getText(); | ||
| } | ||
| } | ||
| return null; |
There was a problem hiding this comment.
what if a null is returned? will it result in exception in the getRunningTaskIds?
There was a problem hiding this comment.
It will not result in an exception in getRunningTaskIds, but it would allow null to be in the result set. I think from this class perspective is valid (if a task exists with a null id, then getRunningTaskIds should include null). And, TaskAnnouncement currently would (should?) exhibit this behaviour on deserialization.
However, I am not clear on the right system-wide behaviour here. If all tasks should have valid Ids, then I think the clearer least-surprising behaviour is to explicitly throw an exception, or maybe filter null elements out of the return value from getRunningTaskIds.
What do you think?
There was a problem hiding this comment.
I don't know if there's a case that there's no id field in the task. Maybe it only exists in our code here in theory. Since here is a performance critical path based on your investigation, I think we can leave the null here.
|
Some minor comments, otherwise looks good. +1 after the above are addressed and CI passes. |
|
HI @samarthjain @xvrl , I believe these comments have all been addressed in a633c74. I included a set of unit tests to confirm that any particular structure should be correctly read, so, I think this is good to go? |
Hi, @jasonk000 - sorry, it looks like I forgot publishing my review comments. They are mostly minor. |
This improves the performance of the ZkWorker task lookup loop by eliminating repeat calls to getRunningTasks() in toImmutable(), and reduces the work performed in isRunningTask() to stream-parse the id field instead of entire JSON blob.
6603e32 to
e41ded2
Compare
|
@samarthjain squashed & rebased |
|
|
||
| import java.util.function.Function; | ||
|
|
||
| public class ZkWorkerTest |
There was a problem hiding this comment.
can we add a test where we serialize the task info using jackson and then deserialize with the task id extractor? this will ensure we update this code whenever the task serialization format changes.
Include a test case that explicitly serializes a TaskAnnouncement and uses the new standalone task extractor.
|
+1. Thank you for your work, @jasonk000 . |
Taking the work done in apache#12096 a little further: 1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner). The original patch only helped with the ZkWorker (RemoteTaskRunner). 2) Improve the ZkWorker version somewhat by avoiding multiple iterations through the task announcements map.
…#14546) * More efficient generation of ImmutableWorkerHolder from WorkerHolder. Taking the work done in #12096 a little further: 1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner). The original patch only helped with the ZkWorker (RemoteTaskRunner). 2) Improve the ZkWorker version somewhat by avoiding multiple iterations through the task announcements map. * Pick better names and use better logic. * Only runnable tasks. * Fix test. * Fix testBlacklistZKWorkers50Percent.
…apache#14546) * More efficient generation of ImmutableWorkerHolder from WorkerHolder. Taking the work done in apache#12096 a little further: 1) Applying a similar optimization to WorkerHolder (HttpRemoteTaskRunner). The original patch only helped with the ZkWorker (RemoteTaskRunner). 2) Improve the ZkWorker version somewhat by avoiding multiple iterations through the task announcements map. * Pick better names and use better logic. * Only runnable tasks. * Fix test. * Fix testBlacklistZKWorkers50Percent.
Description
Improve the performance of
RemoteTaskRunner::tryAssignTaskwhich consumes long periods of CPU on the Overlord during a task restart operation.Screenshot of profiler showing long period of

rtr-pending-..task thread.Screenshot of profile flamegraph for this thread, showing 100pc of CPU in

tryAssignTaskloop:Key changed/added classes in this PR
This change:
getRunningTasks()inZkWorker::toImmutable, and,ZkWorker::isRunningTaskby parsing only theidfield instead of the entire ZkWorker json.By eliminating this extra work, the loop is much tighter.
This is a change coupled to this mailing thread discussion:
https://lists.apache.org/thread/9jgdwrodwsfcg98so6kzfhdmn95gzyrj
Tests
Tests in
RemoteTaskRunner*Test.javacapture this functionality.This PR has: