-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Decrease lock contention in PipelinedStageExecution #14030
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 |
|---|---|---|
|
|
@@ -54,7 +54,10 @@ | |
| import java.util.Set; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.Executor; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
| import java.util.function.Consumer; | ||
| import java.util.stream.Stream; | ||
|
|
||
|
|
@@ -115,12 +118,16 @@ public class PipelinedStageExecution | |
| private final Map<Integer, RemoteTask> tasks = new ConcurrentHashMap<>(); | ||
|
|
||
| // current stage task tracking | ||
| @GuardedBy("this") | ||
| private final ReentrantReadWriteLock allTasksLock = new ReentrantReadWriteLock(); | ||
| @GuardedBy("allTasksLock") | ||
| 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 AtomicLong allTasksCounter = new AtomicLong(); | ||
| private final AtomicLong finishedTasksCounter = new AtomicLong(); | ||
| private final AtomicLong runningTasksCounter = new AtomicLong(); | ||
| private final AtomicBoolean wasFlushingTask = new AtomicBoolean(); | ||
| private final Set<TaskId> finishedTasks = ConcurrentHashMap.newKeySet(); | ||
| private final Set<TaskId> runningTasks = ConcurrentHashMap.newKeySet(); | ||
|
|
||
| // source task tracking | ||
| @GuardedBy("this") | ||
|
|
@@ -224,23 +231,33 @@ public synchronized void schedulingComplete() | |
| return; | ||
| } | ||
|
|
||
| if (isFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| try { | ||
| allTasksLock.readLock().lock(); | ||
| if (isFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| } | ||
| if (isFinished()) { | ||
| stateMachine.transitionToFinished(); | ||
| } | ||
| } | ||
| if (finishedTasks.containsAll(allTasks)) { | ||
| stateMachine.transitionToFinished(); | ||
| finally { | ||
| allTasksLock.readLock().unlock(); | ||
| } | ||
|
|
||
| for (PlanNodeId partitionedSource : stage.getFragment().getPartitionedSources()) { | ||
| schedulingComplete(partitionedSource); | ||
| } | ||
| } | ||
|
|
||
| private synchronized boolean isFlushing() | ||
| private boolean isFlushing() | ||
| { | ||
| // 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)); | ||
| return runningTasksCounter.get() == -1; | ||
|
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. this is super disturbing :) This commit increases complexity a lot (I think above my PR review confidence). I am not convinced we want that.
Contributor
Author
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. Well, the most safe, readable and easy to reason about approach is just to use We have a high lock contention what limits scaling capabilities. In some moment we need to do something with that. I would say that it is expected that lock-free code is more complex that the "lock-full" one. |
||
| } | ||
|
|
||
| private boolean isFinished() | ||
| { | ||
| return finishedTasksCounter.get() == allTasksCounter.get(); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -286,69 +303,81 @@ public synchronized Optional<RemoteTask> scheduleTask( | |
| int partition, | ||
| Multimap<PlanNodeId, Split> initialSplits) | ||
| { | ||
| if (stateMachine.getState().isDone()) { | ||
| return Optional.empty(); | ||
| } | ||
| try { | ||
| allTasksLock.writeLock().lock(); | ||
|
|
||
| checkArgument(!tasks.containsKey(partition), "A task for partition %s already exists", partition); | ||
| if (stateMachine.getState().isDone()) { | ||
| return Optional.empty(); | ||
| } | ||
|
|
||
| OutputBuffers outputBuffers = outputBufferManagers.get(stage.getFragment().getId()).getOutputBuffers(); | ||
| checkArgument(!tasks.containsKey(partition), "A task for partition %s already exists", partition); | ||
|
|
||
| Optional<RemoteTask> optionalTask = stage.createTask( | ||
| node, | ||
| partition, | ||
| attempt, | ||
| bucketToPartition, | ||
| outputBuffers, | ||
| initialSplits, | ||
| ImmutableSet.of(), | ||
| Optional.empty()); | ||
| OutputBuffers outputBuffers = outputBufferManagers.get(stage.getFragment().getId()).getOutputBuffers(); | ||
|
|
||
| if (optionalTask.isEmpty()) { | ||
| return Optional.empty(); | ||
| } | ||
| Optional<RemoteTask> optionalTask = stage.createTask( | ||
| node, | ||
| partition, | ||
| attempt, | ||
| bucketToPartition, | ||
| outputBuffers, | ||
| initialSplits, | ||
| ImmutableSet.of(), | ||
| Optional.empty()); | ||
|
|
||
| RemoteTask task = optionalTask.get(); | ||
| if (optionalTask.isEmpty()) { | ||
| return Optional.empty(); | ||
| } | ||
|
|
||
| tasks.put(partition, task); | ||
| RemoteTask task = optionalTask.get(); | ||
| checkArgument(task.getTaskStatus().getState() != TaskState.FINISHED && task.getTaskStatus().getState() != TaskState.FLUSHING); | ||
|
|
||
| ImmutableMultimap.Builder<PlanNodeId, Split> exchangeSplits = ImmutableMultimap.builder(); | ||
| sourceTasks.forEach((fragmentId, sourceTask) -> { | ||
| TaskStatus status = sourceTask.getTaskStatus(); | ||
| if (status.getState() != TaskState.FINISHED) { | ||
| PlanNodeId planNodeId = exchangeSources.get(fragmentId).getId(); | ||
| exchangeSplits.put(planNodeId, createExchangeSplit(sourceTask, task)); | ||
| } | ||
| }); | ||
| tasks.put(partition, task); | ||
|
|
||
| allTasks.add(task.getTaskId()); | ||
| ImmutableMultimap.Builder<PlanNodeId, Split> exchangeSplits = ImmutableMultimap.builder(); | ||
| sourceTasks.forEach((fragmentId, sourceTask) -> { | ||
| TaskStatus status = sourceTask.getTaskStatus(); | ||
| if (status.getState() != TaskState.FINISHED) { | ||
| PlanNodeId planNodeId = exchangeSources.get(fragmentId).getId(); | ||
| exchangeSplits.put(planNodeId, createExchangeSplit(sourceTask, task)); | ||
| } | ||
| }); | ||
|
|
||
| task.addSplits(exchangeSplits.build()); | ||
| completeSources.forEach(task::noMoreSplits); | ||
| boolean firstTimeAdded = allTasks.add(task.getTaskId()); | ||
| if (firstTimeAdded) { | ||
| allTasksCounter.incrementAndGet(); | ||
| } | ||
| firstTimeAdded = runningTasks.add(task.getTaskId()); | ||
| if (firstTimeAdded) { | ||
| runningTasksCounter.incrementAndGet(); | ||
| } | ||
|
|
||
| task.addSplits(exchangeSplits.build()); | ||
| completeSources.forEach(task::noMoreSplits); | ||
|
|
||
| task.addStateChangeListener(this::updateTaskStatus); | ||
| task.addStateChangeListener(this::updateTaskStatus); | ||
|
|
||
| task.start(); | ||
| task.start(); | ||
|
|
||
| taskLifecycleListener.taskCreated(stage.getFragment().getId(), task); | ||
| taskLifecycleListener.taskCreated(stage.getFragment().getId(), task); | ||
|
|
||
| // update output buffers | ||
| OutputBufferId outputBufferId = new OutputBufferId(task.getTaskId().getPartitionId()); | ||
| updateSourceTasksOutputBuffers(outputBufferManager -> outputBufferManager.addOutputBuffer(outputBufferId)); | ||
| // update output buffers | ||
| OutputBufferId outputBufferId = new OutputBufferId(task.getTaskId().getPartitionId()); | ||
| updateSourceTasksOutputBuffers(outputBufferManager -> outputBufferManager.addOutputBuffer(outputBufferId)); | ||
|
|
||
| return Optional.of(task); | ||
| return Optional.of(task); | ||
| } | ||
| finally { | ||
| allTasksLock.writeLock().unlock(); | ||
| } | ||
| } | ||
|
|
||
| private synchronized void updateTaskStatus(TaskStatus taskStatus) | ||
| private synchronized void updateNotSuccessfulTaskStatus(TaskStatus taskStatus) | ||
| { | ||
| State stageState = stateMachine.getState(); | ||
| if (stageState.isDone()) { | ||
| return; | ||
| } | ||
|
|
||
| TaskState taskState = taskStatus.getState(); | ||
|
|
||
| switch (taskState) { | ||
| switch (taskStatus.getState()) { | ||
| case FAILED: | ||
| RuntimeException failure = taskStatus.getFailures().stream() | ||
| .findFirst() | ||
|
|
@@ -365,25 +394,59 @@ private synchronized void updateTaskStatus(TaskStatus taskStatus) | |
| // A task should only be in the aborted state if the STAGE is done (ABORTED or FAILED) | ||
| fail(new TrinoException(GENERIC_INTERNAL_ERROR, "A task is in the ABORTED state but stage is " + stageState)); | ||
| break; | ||
| case FLUSHING: | ||
| flushingTasks.add(taskStatus.getTaskId()); | ||
| break; | ||
| case FINISHED: | ||
| finishedTasks.add(taskStatus.getTaskId()); | ||
| flushingTasks.remove(taskStatus.getTaskId()); | ||
| break; | ||
| default: | ||
| break; | ||
| } | ||
| } | ||
|
|
||
| private void updateTaskStatus(TaskStatus taskStatus) | ||
| { | ||
| State stageState = stateMachine.getState(); | ||
| TaskState taskState = taskStatus.getState(); | ||
| if (stageState.isDone()) { | ||
| return; | ||
| } | ||
|
|
||
| if (taskState == TaskState.FAILED || taskState == TaskState.CANCELED || taskState == TaskState.ABORTED) { | ||
| updateNotSuccessfulTaskStatus(taskStatus); | ||
| return; | ||
| } | ||
|
|
||
| if (taskState == TaskState.FLUSHING) { | ||
| boolean wasInSet = runningTasks.remove(taskStatus.getTaskId()); | ||
| if (wasInSet) { | ||
| if (!wasFlushingTask.compareAndExchange(false, true)) { | ||
| runningTasksCounter.decrementAndGet(); | ||
| } | ||
| runningTasksCounter.decrementAndGet(); | ||
| } | ||
| } | ||
| else if (taskState == TaskState.FINISHED) { | ||
| boolean addedNow = finishedTasks.add(taskStatus.getTaskId()); | ||
| if (addedNow) { | ||
| finishedTasksCounter.incrementAndGet(); | ||
| } | ||
| boolean wasInSet = runningTasks.remove(taskStatus.getTaskId()); | ||
| if (wasInSet) { | ||
| runningTasksCounter.decrementAndGet(); | ||
| } | ||
| } | ||
|
|
||
| if (stageState == SCHEDULED || stageState == RUNNING || stageState == FLUSHING) { | ||
| if (taskState == TaskState.RUNNING) { | ||
| stateMachine.transitionToRunning(); | ||
| } | ||
| if (isFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| try { | ||
| allTasksLock.readLock().lock(); | ||
| if (isFlushing()) { | ||
| stateMachine.transitionToFlushing(); | ||
| } | ||
| if (isFinished()) { | ||
| stateMachine.transitionToFinished(); | ||
| } | ||
| } | ||
| if (finishedTasks.containsAll(allTasks)) { | ||
| stateMachine.transitionToFinished(); | ||
| finally { | ||
| allTasksLock.readLock().unlock(); | ||
| } | ||
| } | ||
| } | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.