-
Notifications
You must be signed in to change notification settings - Fork 3.1k
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
Reduce lock contention on coordinator #16585
Conversation
91091cb
to
1b0b092
Compare
core/trino-main/src/main/java/io/trino/server/protocol/ExecutingStatementResource.java
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/server/protocol/Query.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java
Outdated
Show resolved
Hide resolved
@@ -228,6 +228,7 @@ private synchronized void sendNextRequest() | |||
|
|||
errorTracker.startRequest(); | |||
future = httpClient.executeAsync(request, createFullJsonResponseHandler(taskInfoCodec)); | |||
lastUpdateNanos.set(System.nanoTime()); |
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.
lastUpdateNanos
is supposed to be set after the request completes and we have the response, not when we're sending a new request. We already enforce the limit that only one request can be in-flight at a time inside of sendNextRequest()
, but if you wanted to reduce the time spent detecting that there's already a request in-flight I see two calls to getTaskInfo()
at the beginning of sendNextRequest()
that could reuse the same returned value.
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.
Callbacks are executed after future is marked as done and lastUpdateNanos
is set within the callback. When a coordinator has high load, this could cause more request to be sent as callback could not be executed before 100ms fixed delay.
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, I see that... maybe we should assign future = null
in the callback after lastUpdateNanos
is set and only send another request if future == null
without checking future.isDone()
to prevent that? Setting lastUpdateNanos
before the request completes could still allow another request to be issued before the callback completes in a similar fashion if the response takes more than 100ms to return.
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.
Setting this future to null in callback would require to add synchronization/AtomicReference.
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.
updateIntervalMillis
is 3 seconds, but scheduleWithFixedDelay
is evaluated at 100ms constant rate. Setting lastUpdateNanos
before send request extends interval for callback evaluation to value of updateIntervalMillis
. This should reduce possibility to have more request in flight.
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.
Repurposing lastUpdateNanos
to throttle back requests in this way makes the code a bit confusing to read. The intent could easily be missed in future changes.
I'm not sure why we have updateScheduledExecutor
blindly trying every 100ms. We could mimic ContinuousTaskStatusFetcher and DynamicFiltersFetcher and schedule the next request during callback processing with a delay of updateIntervalMillis - currentRequestTimeTaken
.
In any case, I would tackle this in a different PR as the rest of the changes are more straightforward.
cc: @sopel39
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 i will address this in other pr
core/trino-main/src/main/java/io/trino/execution/QueryManagerConfig.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
ca00377
to
933d53e
Compare
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
5d31224
to
885474b
Compare
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
a9ed569
to
7c67c7a
Compare
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/metadata/DiscoveryNodeManager.java
Outdated
Show resolved
Hide resolved
be62b78
to
c484ecc
Compare
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.
Latest revision LGTM, although the commit message and PR title explicitly mention Query
's monitor, but there are (perhaps more significant) changes to DiscoveryNodeManager
and SqlStage
. It would also be good to see updated lock contention time stats if possible (and include the updated values in the commit if you have them).
cc: @arhimondr for maintainer review
@@ -208,81 +213,86 @@ public void refreshNodes() | |||
refreshNodesInternal(); | |||
} | |||
|
|||
private synchronized void refreshNodesInternal() | |||
private void refreshNodesInternal() |
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 see any usage of InternalNodeManager in io.trino.server.protocol.Query
and I'm not able to track the lock contention in JFR profiles back to this class.
So I'm not sure why any changes were needed here.
Please share rationale about this or consider dropping these changes if they don't have a sizeable impact.
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.
One execution of query tpcds/q64 is calling TaskCountEstimator
more than 30k times which use getAllNodes
which was synchronized, and it is blocked by refreshNodesInternal
. It rather has minimal impact on blocking time during optimization (but in JFR there were spikes for getAllNodes
), so I will drop this.
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.
We should rather look at the number of nodes in the cluster once at the start of planning for a query and then use that in task count calculations in all optimizer rules. It is probably not a good idea to let each optimizer rule use its own value of active worker nodes for task count estimation. This should be improved separately.
cc: @sopel39 @gaurav8297 @lukasz-stec
core/trino-main/src/main/java/io/trino/server/protocol/ExecutingStatementResource.java
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java
Outdated
Show resolved
Hide resolved
core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java
Outdated
Show resolved
Hide resolved
8bcc86b
to
080b1af
Compare
core/trino-main/src/main/java/io/trino/server/remotetask/TaskInfoFetcher.java
Outdated
Show resolved
Hide resolved
ec0f430
to
1a741a8
Compare
Threads in the application were blocked on locks for a total of 4 h 19 min before this patch and 2 h 59 min after in concurrent benchmark with 40 nodes and 64 queries in parallel.
Thanks for the review @pettyjamesm. Looks good to me. |
Description
Threads in the application were blocked on locks for a total of 4 h 16 min before this patch and 2 h 49 min after in concurrent benchmark with 40 nodes and 64 queries in parallel.
Baseline:
Patch:
Additional context and related issues
Release notes
(X) This is not user-visible or docs only and no release notes are required.
( ) Release notes are required, please propose a release note for me.
( ) Release notes are required, with the following suggested text: