Skip to content
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

Fix a hot loop in DirectExchangeClient #22039

Merged

Conversation

surajkn
Copy link

@surajkn surajkn commented May 20, 2024

Description

The fix changes queuedClients to LinkedHashSet, this helps with the filter condition in scheduleRequestIfNecessary. In addition two for loops are modified to become one loop on Iterator.

Additional context and related issues

We experienced some performance regression as a result of 1ef4427 and this is an attempt to fix the same.

Release notes

( ) This is not user-visible or is docs only, and no release notes are required.
(x ) Release notes are required. Please propose a release note for me.
( ) Release notes are required, with the following suggested text:

# Section
* Fix some things. ({issue}`issuenumber`)

The fix changes queuedClients to LinkedHashSet, this helps with the filter condition
in scheduleRequestIfNecessary. In addition two for loops are modified to become one
loop on Iterator.
Copy link

cla-bot bot commented May 20, 2024

Thank you for your pull request and welcome to the Trino community. We require contributors to sign our Contributor License Agreement, and we don't seem to have you on file. Continue to work with us on the review and improvements in this PR, and submit the signed CLA to [email protected]. Photos, scans, or digitally-signed PDF files are all suitable. Processing may take a few days. The CLA needs to be on file before we merge your changes. For more information, see https://github.com/trinodb/cla

@xkrogen
Copy link
Member

xkrogen commented May 20, 2024

Quick note about the performance regression -- what we observed was that on larger heavily-loaded clusters (hundreds of nodes), this became a severe issue causing cluster-wide degradation in performance.

We've been running this fix on all of our clusters for 9+ months now and have seen full recovery to performance before 1ef4427

@surajkn
Copy link
Author

surajkn commented May 21, 2024

Thank you for your pull request and welcome to the Trino community. We require contributors to sign our Contributor License Agreement, and we don't seem to have you on file. Continue to work with us on the review and improvements in this PR, and submit the signed CLA to [email protected]. Photos, scans, or digitally-signed PDF files are all suitable. Processing may take a few days. The CLA needs to be on file before we merge your changes. For more information, see https://github.com/trinodb/cla

Have sent the completed CLA form to [email protected], waiting for the request to be reviewed and approved

Copy link
Member

@sopel39 sopel39 left a comment

Choose a reason for hiding this comment

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

good catch!. Some comments. Please ping me or re-request review when its ready

@@ -73,7 +73,7 @@ public class DirectExchangeClient
private final Map<URI, HttpPageBufferClient> allClients = new ConcurrentHashMap<>();

@GuardedBy("this")
private final Deque<HttpPageBufferClient> queuedClients = new LinkedList<>();
private final LinkedHashSet<HttpPageBufferClient> queuedClients = new LinkedHashSet<>();
Copy link
Member

Choose a reason for hiding this comment

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

private final Set<HttpPageBufferClient> queuedClients = new LinkedHashSet<>();

Copy link
Author

Choose a reason for hiding this comment

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

Done

@@ -284,17 +284,22 @@ synchronized int scheduleRequestIfNecessary()
long projectedBytesToBeRequested = 0;
int clientCount = 0;

for (HttpPageBufferClient client : queuedClients) {
Iterator<HttpPageBufferClient> queuedClientsIterator = queuedClients.iterator();
Copy link
Member

Choose a reason for hiding this comment

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

nit: queuedClientsIterator -> clientIterator

Copy link
Author

Choose a reason for hiding this comment

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

Done

@@ -304,7 +309,7 @@ public ListenableFuture<Void> isBlocked()
}

@VisibleForTesting
Deque<HttpPageBufferClient> getQueuedClients()
LinkedHashSet<HttpPageBufferClient> getQueuedClients()
Copy link
Member

Choose a reason for hiding this comment

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

Set<HttpPageBufferClient> getQueuedClients()

Copy link
Author

Choose a reason for hiding this comment

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

Done

@cla-bot cla-bot bot added the cla-signed label Jun 3, 2024
@sopel39 sopel39 merged commit 8b6983a into trinodb:master Jun 4, 2024
95 checks passed
@sopel39
Copy link
Member

sopel39 commented Jun 4, 2024

Thanks @surajkn

@sopel39 sopel39 added the no-release-notes This pull request does not require release notes entry label Jun 4, 2024
@github-actions github-actions bot added this to the 450 milestone Jun 4, 2024
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
cla-signed no-release-notes This pull request does not require release notes entry
Development

Successfully merging this pull request may close these issues.

3 participants