Skip to content

perf: improve RemoteTaskRunner task assignment loop performance#12096

Merged
samarthjain merged 2 commits intoapache:masterfrom
jasonk000:zkworker-parsing
Mar 2, 2022
Merged

perf: improve RemoteTaskRunner task assignment loop performance#12096
samarthjain merged 2 commits intoapache:masterfrom
jasonk000:zkworker-parsing

Conversation

@jasonk000
Copy link
Copy Markdown
Contributor

@jasonk000 jasonk000 commented Dec 23, 2021

Description

Improve the performance of RemoteTaskRunner::tryAssignTask which consumes long periods of CPU on the Overlord during a task restart operation.

Screenshot of profiler showing long period of rtr-pending-.. task thread.
image

Screenshot of profile flamegraph for this thread, showing 100pc of CPU in tryAssignTask loop:
image

Key changed/added classes in this PR

This change:

  1. eliminates triple nested call of getRunningTasks() in ZkWorker::toImmutable, and,
  2. reduces the work performed in ZkWorker::isRunningTask by parsing only the id field 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.java capture this functionality.


This PR has:

  • been self-reviewed.
  • been tested in a test Druid cluster (as a part of a larger block of changes).

Comment thread indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java Outdated
@FrankChen021
Copy link
Copy Markdown
Member

Good catch. Overall LGTM. Leave a comment to you to see if we can improve more.

Comment thread indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java Outdated
return parser.getText();
}
}
return null;
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

what if a null is returned? will it result in exception in the getRunningTaskIds?

Copy link
Copy Markdown
Contributor Author

@jasonk000 jasonk000 Dec 24, 2021

Choose a reason for hiding this comment

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

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?

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

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.

Comment thread indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java Outdated
@samarthjain
Copy link
Copy Markdown
Contributor

Some minor comments, otherwise looks good.

+1 after the above are addressed and CI passes.

@jasonk000
Copy link
Copy Markdown
Contributor Author

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?

Comment thread indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java Outdated
Comment thread indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java Outdated
@samarthjain
Copy link
Copy Markdown
Contributor

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.
@jasonk000
Copy link
Copy Markdown
Contributor Author

@samarthjain squashed & rebased


import java.util.function.Function;

public class ZkWorkerTest
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

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.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added test in 9a2175d

Include a test case that explicitly serializes a TaskAnnouncement and
uses the new standalone task extractor.
@samarthjain
Copy link
Copy Markdown
Contributor

+1. Thank you for your work, @jasonk000 .

@samarthjain samarthjain merged commit f594e7a into apache:master Mar 2, 2022
@jasonk000 jasonk000 deleted the zkworker-parsing branch March 2, 2022 17:51
@abhishekagarwal87 abhishekagarwal87 added this to the 0.23.0 milestone May 11, 2022
gianm added a commit to gianm/druid that referenced this pull request Jul 7, 2023
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.
gianm added a commit that referenced this pull request Jul 13, 2023
…#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.
sergioferragut pushed a commit to sergioferragut/druid that referenced this pull request Jul 21, 2023
…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.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants