-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Decrease a lock contention in PipelinedStageExecution #14395
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
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -118,10 +118,8 @@ public class PipelinedStageExecution | |
| // current stage task tracking | ||
| @GuardedBy("this") | ||
| private final Set<TaskId> allTasks = new HashSet<>(); | ||
| @GuardedBy("this") | ||
| private final Set<TaskId> finishedTasks = new HashSet<>(); | ||
| @GuardedBy("this") | ||
| private final Set<TaskId> flushingTasks = new HashSet<>(); | ||
| private final Set<TaskId> finishedTasks = ConcurrentHashMap.newKeySet(); | ||
| private final Set<TaskId> flushingTasks = ConcurrentHashMap.newKeySet(); | ||
|
|
||
| // source task tracking | ||
| @GuardedBy("this") | ||
|
|
@@ -219,16 +217,16 @@ public synchronized void transitionToSchedulingSplits() | |
| } | ||
|
|
||
| @Override | ||
| public synchronized void schedulingComplete() | ||
| public void schedulingComplete() | ||
| { | ||
| if (!stateMachine.transitionToScheduled()) { | ||
| return; | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| } | ||
|
|
||
| if (isFlushing()) { | ||
| if (isStageFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| } | ||
| if (finishedTasks.containsAll(allTasks)) { | ||
| if (isStageFinished()) { | ||
| stateMachine.transitionToFinished(); | ||
| } | ||
|
|
||
|
|
@@ -237,13 +235,6 @@ public synchronized void schedulingComplete() | |
| } | ||
| } | ||
|
|
||
| private synchronized boolean isFlushing() | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| { | ||
| // to transition to flushing, there must be at least one flushing task, and all others must be flushing or finished. | ||
| return !flushingTasks.isEmpty() | ||
| && allTasks.stream().allMatch(taskId -> finishedTasks.contains(taskId) || flushingTasks.contains(taskId)); | ||
| } | ||
|
|
||
| @Override | ||
| public synchronized void schedulingComplete(PlanNodeId partitionedSource) | ||
| { | ||
|
|
@@ -340,13 +331,13 @@ public synchronized Optional<RemoteTask> scheduleTask( | |
| return Optional.of(task); | ||
| } | ||
|
|
||
| private synchronized void updateTaskStatus(TaskStatus taskStatus) | ||
| private void updateTaskStatus(TaskStatus taskStatus) | ||
| { | ||
| State stageState = stateMachine.getState(); | ||
| if (stageState.isDone()) { | ||
| return; | ||
| } | ||
|
|
||
| boolean newFlushingOrFinishedTaskObserved = false; | ||
| TaskState taskState = taskStatus.getState(); | ||
|
|
||
| switch (taskState) { | ||
|
|
@@ -367,11 +358,10 @@ private synchronized void updateTaskStatus(TaskStatus taskStatus) | |
| fail(new TrinoException(GENERIC_INTERNAL_ERROR, "A task is in the ABORTED state but stage is " + stageState)); | ||
| break; | ||
| case FLUSHING: | ||
| flushingTasks.add(taskStatus.getTaskId()); | ||
| newFlushingOrFinishedTaskObserved = addFlushingTask(taskStatus.getTaskId()); | ||
| break; | ||
| case FINISHED: | ||
| finishedTasks.add(taskStatus.getTaskId()); | ||
| flushingTasks.remove(taskStatus.getTaskId()); | ||
| newFlushingOrFinishedTaskObserved = addFinishedTask(taskStatus.getTaskId()); | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| break; | ||
| default: | ||
| } | ||
|
|
@@ -380,13 +370,54 @@ private synchronized void updateTaskStatus(TaskStatus taskStatus) | |
| if (taskState == TaskState.RUNNING) { | ||
| stateMachine.transitionToRunning(); | ||
| } | ||
| if (isFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| // avoid extra synchronization if no new flushing or finished task was observed | ||
| if (newFlushingOrFinishedTaskObserved) { | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yup. You are correct. |
||
| if (isStageFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| } | ||
| if (isStageFinished()) { | ||
| stateMachine.transitionToFinished(); | ||
| } | ||
| } | ||
| if (finishedTasks.containsAll(allTasks)) { | ||
| stateMachine.transitionToFinished(); | ||
| } | ||
| } | ||
|
|
||
| private synchronized boolean isStageFlushing() | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| { | ||
| // to transition to flushing, there must be at least one flushing task, and all others must be flushing or finished. | ||
| return !flushingTasks.isEmpty() | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| && allTasks.stream().allMatch(taskId -> finishedTasks.contains(taskId) || flushingTasks.contains(taskId)); | ||
| } | ||
|
|
||
| private synchronized boolean isStageFinished() | ||
| { | ||
| return finishedTasks.containsAll(allTasks); | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| } | ||
|
|
||
| private boolean addFlushingTask(TaskId taskId) | ||
| { | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| if (!flushingTasks.contains(taskId) && !finishedTasks.contains(taskId)) { | ||
| synchronized (this) { | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| // We need to check whether that task is not already finished. It could happen because of out of order of | ||
| // task status events | ||
| if (!finishedTasks.contains(taskId)) { | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| return flushingTasks.add(taskId); | ||
| } | ||
| } | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| private boolean addFinishedTask(TaskId taskId) | ||
| { | ||
| if (!finishedTasks.contains(taskId)) { | ||
| synchronized (this) { | ||
| boolean added = finishedTasks.add(taskId); | ||
| flushingTasks.remove(taskId); | ||
|
radek-kondziolka marked this conversation as resolved.
Outdated
|
||
| return added; | ||
| } | ||
| } | ||
| return false; | ||
| } | ||
|
|
||
| private ExecutionFailureInfo rewriteTransportFailure(ExecutionFailureInfo executionFailureInfo) | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.