Support limiting the number of unacknowledged source splits per task#15761
Conversation
ad79842 to
5ec261c
Compare
5ec261c to
88a5b11
Compare
shixuan-fan
left a comment
There was a problem hiding this comment.
"Refactor NodeAssignmentStats to simplify the tracked representation" LGTM
There was a problem hiding this comment.
It feels a bit weird to have getQueuedSplitCount return queuedSplitCount + assignedSplits, but I don't have a better name in mind.
There was a problem hiding this comment.
Yeah, the structure here is a little quirky (but is fundamentally the same as before, just with 1 fewer HashMap). Any split assigned within the current batch is "effectively queued" even though it hasn't made its way to the task yet. I couldn't think of a name that worked any better at making that piece clear.
shixuan-fan
left a comment
There was a problem hiding this comment.
"Avoid unnecessary node list sorting in SimpleNodeSelector"
.../src/main/java/com/facebook/presto/execution/scheduler/nodeSelection/SimpleNodeSelector.java
Outdated
Show resolved
Hide resolved
shixuan-fan
left a comment
There was a problem hiding this comment.
"Support limiting the number of unacknowledged splits per task"
LGTM, mostly questions for my own understanding
presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeAssignmentStats.java
Outdated
Show resolved
Hide resolved
presto-main/src/main/java/com/facebook/presto/execution/scheduler/NodeScheduler.java
Outdated
Show resolved
Hide resolved
There was a problem hiding this comment.
Just curious, what is the rationale behind using 500 as the default value? 🤔
There was a problem hiding this comment.
In an experimental setup I was running with differently (very large) split queue depth configurations and small files (aka: cheap splits that are processed very quickly once delivered), 500 seemed to be about the point of diminishing returns in terms of setting this value higher.
In the original iteration, I had this default to Integer.MAX_VALUE since I don't aim to affect any of the default scheduling behavior with this change, but rather just want to add a safety net for deeper split queues blowing up task update sizes. Dain convinced me to pick a more "reasonable" value that was still above the ceiling of having any affect on the current default configs.
There was a problem hiding this comment.
It might be a dumb question but I'm curious to understand why we need to check this before listeners are registered?
There was a problem hiding this comment.
It's possible for task initial splits to already exceed the pending split limit inside of the constructor, so setting the "no space" flag appropriately at initialization time had some effect but... now that I'm looking I can't see what the effect was. It might have had some relationship to the MockRemoteTask implementation in test and maybe isn't strictly required here.
88a5b11 to
545d8b9
Compare
Adds support for tracking (and optionally limiting) the number of splits that are not yet acknowledged for a given task. Previously, the only two scheduler supported limits were the number of total splits across all tasks on the worker node and the number of splits queued for a given task which included: - splits received by the worker but not yet running as of the last task status update - splits queued in the coordinator local RemoteTask but not yet sent to the task - splits assigned in the current scheduling run but not yet added to the coordinator local RemoteTask The new max_unacknowledged_splits_per_task session property enables setting a limit on the number of splits that are either: - queued on the coordinator-local RemoteTask but not yet sent or at least confirmed to have been received by the worker - assigned to the task in the current scheduling batch but not yet added to the coordinator-local RemoteTask This limit enforcement takes precedence over both of the other existing split limit configurations and is designed to prevent large task update requests that might cause a query to fail.
545d8b9 to
88049f2
Compare
|
cc @NikhilCollooru This might touch some of the codes that you are currently working on. I don't think there is conflict but want to give you a heads-up. |
Adds support for tracking (and optionally limiting) the number of splits that are not yet acknowledged for a given task. Previously, the only two scheduler supported limits were the number of total splits across all tasks on the worker node and the number of splits queued for a given task which included:
RemoteTaskbut not yet sent to the taskRemoteTaskThe new
max_unacknowledged_splits_per_tasksession property enables setting a limit on the number of splits that are either:RemoteTaskbut not yet sent or at least not confirmed to have been received by the workerRemoteTaskThis limit enforcement takes precedence over both of the other existing split limit configurations and is designed to prevent large task update requests that might cause a query to fail.