diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java
index 66c841b5d894..ec6c2b041105 100644
--- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java
+++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskManager.java
@@ -40,9 +40,9 @@
import io.trino.execution.buffer.BufferResult;
import io.trino.execution.buffer.OutputBuffers;
import io.trino.execution.buffer.PipelinedOutputBuffers;
-import io.trino.execution.executor.PrioritizedSplitRunner;
+import io.trino.execution.executor.RunningSplitInfo;
import io.trino.execution.executor.TaskExecutor;
-import io.trino.execution.executor.TaskExecutor.RunningSplitInfo;
+import io.trino.execution.executor.timesharing.PrioritizedSplitRunner;
import io.trino.memory.LocalMemoryManager;
import io.trino.memory.NodeMemoryConfig;
import io.trino.memory.QueryContext;
@@ -90,7 +90,7 @@
import static io.trino.SystemSessionProperties.resourceOvercommit;
import static io.trino.cache.SafeCaches.buildNonEvictableCache;
import static io.trino.execution.SqlTask.createSqlTask;
-import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
+import static io.trino.execution.executor.timesharing.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
import static io.trino.operator.RetryPolicy.TASK;
import static io.trino.spi.StandardErrorCode.ABANDONED_TASK;
import static io.trino.spi.StandardErrorCode.GENERIC_USER_ERROR;
diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java
index ddd2fb50aa09..6758e362ba88 100644
--- a/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java
+++ b/core/trino-main/src/main/java/io/trino/execution/TaskManagerConfig.java
@@ -45,6 +45,7 @@
"task.level-absolute-priority"})
public class TaskManagerConfig
{
+ private boolean threadPerDriverSchedulerEnabled;
private boolean perOperatorCpuTimerEnabled = true;
private boolean taskCpuTimerEnabled = true;
private boolean statisticsCpuTimerEnabled = true;
@@ -107,6 +108,18 @@ public class TaskManagerConfig
private BigDecimal levelTimeMultiplier = new BigDecimal(2.0);
+ @Config("experimental.thread-per-driver-scheduler-enabled")
+ public TaskManagerConfig setThreadPerDriverSchedulerEnabled(boolean enabled)
+ {
+ this.threadPerDriverSchedulerEnabled = enabled;
+ return this;
+ }
+
+ public boolean isThreadPerDriverSchedulerEnabled()
+ {
+ return threadPerDriverSchedulerEnabled;
+ }
+
@MinDuration("1ms")
@MaxDuration("10s")
@NotNull
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/RunningSplitInfo.java b/core/trino-main/src/main/java/io/trino/execution/executor/RunningSplitInfo.java
new file mode 100644
index 000000000000..6669d7ef597b
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/RunningSplitInfo.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor;
+
+import com.google.common.collect.ComparisonChain;
+import io.trino.execution.TaskId;
+
+import java.util.function.Supplier;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A class representing a split that is running on the TaskRunner.
+ * It has a Thread object that gets assigned while assigning the split
+ * to the taskRunner. However, when the TaskRunner moves to a different split,
+ * the thread stored here will not remain assigned to this split anymore.
+ */
+public class RunningSplitInfo
+ implements Comparable
+{
+ private final long startTime;
+ private final String threadId;
+ private final Thread thread;
+ private boolean printed;
+ private final TaskId taskId;
+ private final Supplier splitInfo;
+
+ public RunningSplitInfo(long startTime, String threadId, Thread thread, TaskId taskId, Supplier splitInfo)
+ {
+ this.startTime = startTime;
+ this.threadId = requireNonNull(threadId, "threadId is null");
+ this.thread = requireNonNull(thread, "thread is null");
+ this.taskId = requireNonNull(taskId, "taskId is null");
+ this.splitInfo = requireNonNull(splitInfo, "split is null");
+ this.printed = false;
+ }
+
+ public long getStartTime()
+ {
+ return startTime;
+ }
+
+ public String getThreadId()
+ {
+ return threadId;
+ }
+
+ public Thread getThread()
+ {
+ return thread;
+ }
+
+ public TaskId getTaskId()
+ {
+ return taskId;
+ }
+
+ /**
+ * {@link PrioritizedSplitRunner#getInfo()} provides runtime statistics for the split (such as total cpu utilization so far).
+ * A value returned from this method changes over time and cannot be cached as a field of {@link RunningSplitInfo}.
+ *
+ * @return Formatted string containing runtime statistics for the split.
+ */
+ public String getSplitInfo()
+ {
+ return splitInfo.get();
+ }
+
+ public boolean isPrinted()
+ {
+ return printed;
+ }
+
+ public void setPrinted()
+ {
+ printed = true;
+ }
+
+ @Override
+ public int compareTo(RunningSplitInfo o)
+ {
+ return ComparisonChain.start()
+ .compare(startTime, o.getStartTime())
+ .compare(threadId, o.getThreadId())
+ .result();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/TaskExecutor.java b/core/trino-main/src/main/java/io/trino/execution/executor/TaskExecutor.java
index f7ed83aa7b7e..d9ddc32772be 100644
--- a/core/trino-main/src/main/java/io/trino/execution/executor/TaskExecutor.java
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/TaskExecutor.java
@@ -13,1000 +13,33 @@
*/
package io.trino.execution.executor;
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Ticker;
-import com.google.common.collect.ComparisonChain;
import com.google.common.util.concurrent.ListenableFuture;
-import com.google.errorprone.annotations.ThreadSafe;
-import com.google.errorprone.annotations.concurrent.GuardedBy;
-import com.google.inject.Inject;
-import io.airlift.concurrent.SetThreadName;
-import io.airlift.concurrent.ThreadPoolExecutorMBean;
-import io.airlift.log.Logger;
-import io.airlift.stats.CounterStat;
-import io.airlift.stats.DistributionStat;
-import io.airlift.stats.TimeDistribution;
-import io.airlift.stats.TimeStat;
import io.airlift.units.Duration;
-import io.opentelemetry.api.trace.Span;
-import io.opentelemetry.api.trace.Tracer;
-import io.opentelemetry.context.Context;
import io.trino.execution.SplitRunner;
import io.trino.execution.TaskId;
-import io.trino.execution.TaskManagerConfig;
-import io.trino.spi.TrinoException;
-import io.trino.spi.VersionEmbedder;
-import io.trino.tracing.TrinoAttributes;
-import jakarta.annotation.PostConstruct;
-import jakarta.annotation.PreDestroy;
-import org.weakref.jmx.Managed;
-import org.weakref.jmx.Nested;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
import java.util.List;
-import java.util.Map;
import java.util.OptionalInt;
import java.util.Set;
-import java.util.SortedSet;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicLongArray;
import java.util.function.DoubleSupplier;
import java.util.function.Predicate;
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkState;
-import static com.google.common.collect.ImmutableSet.toImmutableSet;
-import static com.google.common.collect.Sets.newConcurrentHashSet;
-import static io.airlift.concurrent.Threads.threadsNamed;
-import static io.airlift.tracing.Tracing.noopTracer;
-import static io.trino.execution.executor.MultilevelSplitQueue.computeLevel;
-import static io.trino.version.EmbedVersion.testingVersionEmbedder;
-import static java.lang.Math.min;
-import static java.lang.String.format;
-import static java.util.Objects.requireNonNull;
-import static java.util.concurrent.Executors.newCachedThreadPool;
-import static java.util.concurrent.TimeUnit.MICROSECONDS;
-import static java.util.concurrent.TimeUnit.NANOSECONDS;
-
-@ThreadSafe
-public class TaskExecutor
+public interface TaskExecutor
{
- private static final Logger log = Logger.get(TaskExecutor.class);
- private static final AtomicLong NEXT_RUNNER_ID = new AtomicLong();
-
- private final ExecutorService executor;
- private final ThreadPoolExecutorMBean executorMBean;
-
- private final int runnerThreads;
- private final int minimumNumberOfDrivers;
- private final int guaranteedNumberOfDriversPerTask;
- private final int maximumNumberOfDriversPerTask;
- private final VersionEmbedder versionEmbedder;
- private final Tracer tracer;
-
- private final Ticker ticker;
-
- private final Duration stuckSplitsWarningThreshold;
- private final SortedSet runningSplitInfos = new ConcurrentSkipListSet<>();
-
- @GuardedBy("this")
- private final List tasks;
-
- /**
- * All splits registered with the task executor.
- */
- @GuardedBy("this")
- private final Set allSplits = new HashSet<>();
-
- /**
- * Intermediate splits (i.e. splits that should not be queued).
- */
- @GuardedBy("this")
- private final Set intermediateSplits = new HashSet<>();
-
- /**
- * Splits waiting for a runner thread.
- */
- private final MultilevelSplitQueue waitingSplits;
-
- /**
- * Splits running on a thread.
- */
- private final Set runningSplits = newConcurrentHashSet();
-
- /**
- * Splits blocked by the driver.
- */
- private final Map> blockedSplits = new ConcurrentHashMap<>();
-
- private final AtomicLongArray completedTasksPerLevel = new AtomicLongArray(5);
- private final AtomicLongArray completedSplitsPerLevel = new AtomicLongArray(5);
-
- private final TimeStat splitQueuedTime = new TimeStat(NANOSECONDS);
- private final TimeStat splitWallTime = new TimeStat(NANOSECONDS);
-
- private final TimeDistribution leafSplitWallTime = new TimeDistribution(MICROSECONDS);
- private final TimeDistribution intermediateSplitWallTime = new TimeDistribution(MICROSECONDS);
-
- private final TimeDistribution leafSplitScheduledTime = new TimeDistribution(MICROSECONDS);
- private final TimeDistribution intermediateSplitScheduledTime = new TimeDistribution(MICROSECONDS);
-
- private final TimeDistribution leafSplitWaitTime = new TimeDistribution(MICROSECONDS);
- private final TimeDistribution intermediateSplitWaitTime = new TimeDistribution(MICROSECONDS);
-
- private final TimeDistribution leafSplitCpuTime = new TimeDistribution(MICROSECONDS);
- private final TimeDistribution intermediateSplitCpuTime = new TimeDistribution(MICROSECONDS);
-
- // shared between SplitRunners
- private final CounterStat globalCpuTimeMicros = new CounterStat();
- private final CounterStat globalScheduledTimeMicros = new CounterStat();
-
- private final TimeStat blockedQuantaWallTime = new TimeStat(MICROSECONDS);
- private final TimeStat unblockedQuantaWallTime = new TimeStat(MICROSECONDS);
-
- private final DistributionStat leafSplitsSize = new DistributionStat();
- @GuardedBy("this")
- private long lastLeafSplitsSizeRecordTime;
- @GuardedBy("this")
- private long lastLeafSplitsSize;
-
- private volatile boolean closed;
-
- @Inject
- public TaskExecutor(TaskManagerConfig config, VersionEmbedder versionEmbedder, Tracer tracer, MultilevelSplitQueue splitQueue)
- {
- this(
- config.getMaxWorkerThreads(),
- config.getMinDrivers(),
- config.getMinDriversPerTask(),
- config.getMaxDriversPerTask(),
- config.getInterruptStuckSplitTasksWarningThreshold(),
- versionEmbedder,
- tracer,
- splitQueue,
- Ticker.systemTicker());
- }
-
- @VisibleForTesting
- public TaskExecutor(int runnerThreads, int minDrivers, int guaranteedNumberOfDriversPerTask, int maximumNumberOfDriversPerTask, Ticker ticker)
- {
- this(runnerThreads, minDrivers, guaranteedNumberOfDriversPerTask, maximumNumberOfDriversPerTask, new Duration(10, TimeUnit.MINUTES), testingVersionEmbedder(), noopTracer(), new MultilevelSplitQueue(2), ticker);
- }
-
- @VisibleForTesting
- public TaskExecutor(int runnerThreads, int minDrivers, int guaranteedNumberOfDriversPerTask, int maximumNumberOfDriversPerTask, MultilevelSplitQueue splitQueue, Ticker ticker)
- {
- this(runnerThreads, minDrivers, guaranteedNumberOfDriversPerTask, maximumNumberOfDriversPerTask, new Duration(10, TimeUnit.MINUTES), testingVersionEmbedder(), noopTracer(), splitQueue, ticker);
- }
-
- @VisibleForTesting
- public TaskExecutor(
- int runnerThreads,
- int minDrivers,
- int guaranteedNumberOfDriversPerTask,
- int maximumNumberOfDriversPerTask,
- Duration stuckSplitsWarningThreshold,
- VersionEmbedder versionEmbedder,
- Tracer tracer,
- MultilevelSplitQueue splitQueue,
- Ticker ticker)
- {
- checkArgument(runnerThreads > 0, "runnerThreads must be at least 1");
- checkArgument(guaranteedNumberOfDriversPerTask > 0, "guaranteedNumberOfDriversPerTask must be at least 1");
- checkArgument(maximumNumberOfDriversPerTask > 0, "maximumNumberOfDriversPerTask must be at least 1");
- checkArgument(guaranteedNumberOfDriversPerTask <= maximumNumberOfDriversPerTask, "guaranteedNumberOfDriversPerTask cannot be greater than maximumNumberOfDriversPerTask");
-
- // we manage thread pool size directly, so create an unlimited pool
- this.executor = newCachedThreadPool(threadsNamed("task-processor-%s"));
- this.executorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) executor);
- this.runnerThreads = runnerThreads;
- this.versionEmbedder = requireNonNull(versionEmbedder, "versionEmbedder is null");
- this.tracer = requireNonNull(tracer, "tracer is null");
-
- this.ticker = requireNonNull(ticker, "ticker is null");
- this.stuckSplitsWarningThreshold = requireNonNull(stuckSplitsWarningThreshold, "stuckSplitsWarningThreshold is null");
-
- this.minimumNumberOfDrivers = minDrivers;
- this.guaranteedNumberOfDriversPerTask = guaranteedNumberOfDriversPerTask;
- this.maximumNumberOfDriversPerTask = maximumNumberOfDriversPerTask;
- this.waitingSplits = requireNonNull(splitQueue, "splitQueue is null");
- this.tasks = new LinkedList<>();
- this.lastLeafSplitsSizeRecordTime = ticker.read();
- }
-
- @PostConstruct
- public synchronized void start()
- {
- checkState(!closed, "TaskExecutor is closed");
- for (int i = 0; i < runnerThreads; i++) {
- addRunnerThread();
- }
- }
-
- @PreDestroy
- public synchronized void stop()
- {
- closed = true;
- executor.shutdownNow();
- }
-
- @Override
- public synchronized String toString()
- {
- return toStringHelper(this)
- .add("runnerThreads", runnerThreads)
- .add("allSplits", allSplits.size())
- .add("intermediateSplits", intermediateSplits.size())
- .add("waitingSplits", waitingSplits.size())
- .add("runningSplits", runningSplits.size())
- .add("blockedSplits", blockedSplits.size())
- .toString();
- }
-
- private synchronized void addRunnerThread()
- {
- try {
- executor.execute(versionEmbedder.embedVersion(new TaskRunner()));
- }
- catch (RejectedExecutionException ignored) {
- }
- }
-
- public synchronized TaskHandle addTask(
+ TaskHandle addTask(
TaskId taskId,
DoubleSupplier utilizationSupplier,
int initialSplitConcurrency,
Duration splitConcurrencyAdjustFrequency,
- OptionalInt maxDriversPerTask)
- {
- requireNonNull(taskId, "taskId is null");
- requireNonNull(utilizationSupplier, "utilizationSupplier is null");
- checkArgument(maxDriversPerTask.isEmpty() || maxDriversPerTask.getAsInt() <= maximumNumberOfDriversPerTask,
- "maxDriversPerTask cannot be greater than the configured value");
-
- log.debug("Task scheduled %s", taskId);
-
- TaskHandle taskHandle = new TaskHandle(taskId, waitingSplits, utilizationSupplier, initialSplitConcurrency, splitConcurrencyAdjustFrequency, maxDriversPerTask);
-
- tasks.add(taskHandle);
- return taskHandle;
- }
-
- public void removeTask(TaskHandle taskHandle)
- {
- try (SetThreadName ignored = new SetThreadName("Task-%s", taskHandle.getTaskId())) {
- // Skip additional scheduling if the task was already destroyed
- if (!doRemoveTask(taskHandle)) {
- return;
- }
- }
-
- // replace blocked splits that were terminated
- synchronized (this) {
- addNewEntrants();
- recordLeafSplitsSize();
- }
- }
-
- /**
- * Returns true if the task handle was destroyed and removed splits as a result that may need to be replaced. Otherwise,
- * if the {@link TaskHandle} was already destroyed or no splits were removed then this method returns false and no additional
- * splits need to be scheduled.
- */
- private boolean doRemoveTask(TaskHandle taskHandle)
- {
- List splits;
- synchronized (this) {
- tasks.remove(taskHandle);
-
- // Task is already destroyed
- if (taskHandle.isDestroyed()) {
- return false;
- }
-
- splits = taskHandle.destroy();
- // stop tracking splits (especially blocked splits which may never unblock)
- allSplits.removeAll(splits);
- intermediateSplits.removeAll(splits);
- blockedSplits.keySet().removeAll(splits);
- waitingSplits.removeAll(splits);
- recordLeafSplitsSize();
- }
-
- // call destroy outside of synchronized block as it is expensive and doesn't need a lock on the task executor
- for (PrioritizedSplitRunner split : splits) {
- split.destroy();
- }
-
- // record completed stats
- long threadUsageNanos = taskHandle.getScheduledNanos();
- completedTasksPerLevel.incrementAndGet(computeLevel(threadUsageNanos));
-
- log.debug("Task finished or failed %s", taskHandle.getTaskId());
- return !splits.isEmpty();
- }
-
- public List> enqueueSplits(TaskHandle taskHandle, boolean intermediate, List extends SplitRunner> taskSplits)
- {
- List splitsToDestroy = new ArrayList<>();
- List> finishedFutures = new ArrayList<>(taskSplits.size());
- synchronized (this) {
- for (SplitRunner taskSplit : taskSplits) {
- TaskId taskId = taskHandle.getTaskId();
- int splitId = taskHandle.getNextSplitId();
-
- Span splitSpan = tracer.spanBuilder(intermediate ? "split (intermediate)" : "split (leaf)")
- .setParent(Context.current().with(taskSplit.getPipelineSpan()))
- .setAttribute(TrinoAttributes.QUERY_ID, taskId.getQueryId().toString())
- .setAttribute(TrinoAttributes.STAGE_ID, taskId.getStageId().toString())
- .setAttribute(TrinoAttributes.TASK_ID, taskId.toString())
- .setAttribute(TrinoAttributes.PIPELINE_ID, taskId.getStageId() + "-" + taskSplit.getPipelineId())
- .setAttribute(TrinoAttributes.SPLIT_ID, taskId + "-" + splitId)
- .startSpan();
-
- PrioritizedSplitRunner prioritizedSplitRunner = new PrioritizedSplitRunner(
- taskHandle,
- splitId,
- taskSplit,
- splitSpan,
- tracer,
- ticker,
- globalCpuTimeMicros,
- globalScheduledTimeMicros,
- blockedQuantaWallTime,
- unblockedQuantaWallTime);
-
- if (intermediate) {
- // add the runner to the handle so it can be destroyed if the task is canceled
- if (taskHandle.recordIntermediateSplit(prioritizedSplitRunner)) {
- // Note: we do not record queued time for intermediate splits
- startIntermediateSplit(prioritizedSplitRunner);
- }
- else {
- splitsToDestroy.add(prioritizedSplitRunner);
- }
- }
- else {
- // add this to the work queue for the task
- if (taskHandle.enqueueSplit(prioritizedSplitRunner)) {
- // if task is under the limit for guaranteed splits, start one
- scheduleTaskIfNecessary(taskHandle);
- // if globally we have more resources, start more
- addNewEntrants();
- }
- else {
- splitsToDestroy.add(prioritizedSplitRunner);
- }
- }
-
- finishedFutures.add(prioritizedSplitRunner.getFinishedFuture());
- }
- recordLeafSplitsSize();
- }
- for (PrioritizedSplitRunner split : splitsToDestroy) {
- split.destroy();
- }
- return finishedFutures;
- }
-
- private void splitFinished(PrioritizedSplitRunner split)
- {
- completedSplitsPerLevel.incrementAndGet(split.getPriority().getLevel());
- synchronized (this) {
- allSplits.remove(split);
-
- long wallNanos = System.nanoTime() - split.getCreatedNanos();
- splitWallTime.add(Duration.succinctNanos(wallNanos));
-
- if (intermediateSplits.remove(split)) {
- intermediateSplitWallTime.add(wallNanos);
- intermediateSplitScheduledTime.add(split.getScheduledNanos());
- intermediateSplitWaitTime.add(split.getWaitNanos());
- intermediateSplitCpuTime.add(split.getCpuTimeNanos());
- }
- else {
- leafSplitWallTime.add(wallNanos);
- leafSplitScheduledTime.add(split.getScheduledNanos());
- leafSplitWaitTime.add(split.getWaitNanos());
- leafSplitCpuTime.add(split.getCpuTimeNanos());
- }
-
- TaskHandle taskHandle = split.getTaskHandle();
- taskHandle.splitComplete(split);
-
- scheduleTaskIfNecessary(taskHandle);
-
- addNewEntrants();
- recordLeafSplitsSize();
- }
- // call destroy outside of synchronized block as it is expensive and doesn't need a lock on the task executor
- split.destroy();
- }
-
- private synchronized void scheduleTaskIfNecessary(TaskHandle taskHandle)
- {
- // if task has less than the minimum guaranteed splits running,
- // immediately schedule new splits for this task. This assures
- // that a task gets its fair amount of consideration (you have to
- // have splits to be considered for running on a thread).
- int splitsToSchedule = min(guaranteedNumberOfDriversPerTask, taskHandle.getMaxDriversPerTask().orElse(Integer.MAX_VALUE)) - taskHandle.getRunningLeafSplits();
- for (int i = 0; i < splitsToSchedule; ++i) {
- PrioritizedSplitRunner split = taskHandle.pollNextSplit();
- if (split == null) {
- // no more splits to schedule
- return;
- }
-
- startSplit(split);
- splitQueuedTime.add(Duration.nanosSince(split.getCreatedNanos()));
- }
- recordLeafSplitsSize();
- }
-
- private synchronized void addNewEntrants()
- {
- // Ignore intermediate splits when checking minimumNumberOfDrivers.
- // Otherwise with (for example) minimumNumberOfDrivers = 100, 200 intermediate splits
- // and 100 leaf splits, depending on order of appearing splits, number of
- // simultaneously running splits may vary. If leaf splits start first, there will
- // be 300 running splits. If intermediate splits start first, there will be only
- // 200 running splits.
- int running = allSplits.size() - intermediateSplits.size();
- for (int i = 0; i < minimumNumberOfDrivers - running; i++) {
- PrioritizedSplitRunner split = pollNextSplitWorker();
- if (split == null) {
- break;
- }
-
- splitQueuedTime.add(Duration.nanosSince(split.getCreatedNanos()));
- startSplit(split);
- }
- }
-
- private synchronized void startIntermediateSplit(PrioritizedSplitRunner split)
- {
- startSplit(split);
- intermediateSplits.add(split);
- }
-
- private synchronized void startSplit(PrioritizedSplitRunner split)
- {
- allSplits.add(split);
- waitingSplits.offer(split);
- }
-
- private synchronized PrioritizedSplitRunner pollNextSplitWorker()
- {
- // todo find a better algorithm for this
- // find the first task that produces a split, then move that task to the
- // end of the task list, so we get round robin
- for (Iterator iterator = tasks.iterator(); iterator.hasNext(); ) {
- TaskHandle task = iterator.next();
- // skip tasks that are already running the configured max number of drivers
- if (task.getRunningLeafSplits() >= task.getMaxDriversPerTask().orElse(maximumNumberOfDriversPerTask)) {
- continue;
- }
- PrioritizedSplitRunner split = task.pollNextSplit();
- if (split != null) {
- // move task to end of list
- iterator.remove();
-
- // CAUTION: we are modifying the list in the loop which would normally
- // cause a ConcurrentModificationException but we exit immediately
- tasks.add(task);
- return split;
- }
- }
- return null;
- }
-
- private synchronized void recordLeafSplitsSize()
- {
- long now = ticker.read();
- long timeDifference = now - this.lastLeafSplitsSizeRecordTime;
- if (timeDifference > 0) {
- this.leafSplitsSize.add(lastLeafSplitsSize, timeDifference);
- this.lastLeafSplitsSizeRecordTime = now;
- }
- // always record new lastLeafSplitsSize as it might have changed
- // even if timeDifference is 0
- this.lastLeafSplitsSize = allSplits.size() - intermediateSplits.size();
- }
-
- private class TaskRunner
- implements Runnable
- {
- private final long runnerId = NEXT_RUNNER_ID.getAndIncrement();
-
- @Override
- public void run()
- {
- try (SetThreadName runnerName = new SetThreadName("SplitRunner-%s", runnerId)) {
- while (!closed && !Thread.currentThread().isInterrupted()) {
- // select next worker
- PrioritizedSplitRunner split;
- try {
- split = waitingSplits.take();
- }
- catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- return;
- }
-
- String threadId = split.getTaskHandle().getTaskId() + "-" + split.getSplitId();
- try (SetThreadName splitName = new SetThreadName(threadId)) {
- RunningSplitInfo splitInfo = new RunningSplitInfo(ticker.read(), threadId, Thread.currentThread(), split);
- runningSplitInfos.add(splitInfo);
- runningSplits.add(split);
-
- ListenableFuture blocked;
- try {
- blocked = split.process();
- }
- finally {
- runningSplitInfos.remove(splitInfo);
- runningSplits.remove(split);
- }
-
- if (split.isFinished()) {
- if (log.isDebugEnabled()) {
- log.debug("%s is finished", split.getInfo());
- }
- splitFinished(split);
- }
- else {
- if (blocked.isDone()) {
- waitingSplits.offer(split);
- }
- else {
- blockedSplits.put(split, blocked);
- blocked.addListener(() -> {
- blockedSplits.remove(split);
- // reset the level priority to prevent previously-blocked splits from starving existing splits
- split.resetLevelPriority();
- waitingSplits.offer(split);
- }, executor);
- }
- }
- }
- catch (Throwable t) {
- // ignore random errors due to driver thread interruption
- if (!split.isDestroyed()) {
- if (t instanceof TrinoException trinoException) {
- log.error(t, "Error processing %s: %s: %s", split.getInfo(), trinoException.getErrorCode().getName(), trinoException.getMessage());
- }
- else {
- log.error(t, "Error processing %s", split.getInfo());
- }
- }
- splitFinished(split);
- }
- finally {
- // Clear the interrupted flag on the current thread, driver cancellation may have triggered an interrupt
- if (Thread.interrupted()) {
- if (closed) {
- // reset interrupted flag if closed before interrupt
- Thread.currentThread().interrupt();
- }
- }
- }
- }
- }
- finally {
- // unless we have been closed, we need to replace this thread
- if (!closed) {
- addRunnerThread();
- }
- }
- }
- }
-
- //
- // STATS
- //
-
- @Managed
- public synchronized int getTasks()
- {
- return tasks.size();
- }
-
- @Managed
- public int getRunnerThreads()
- {
- return runnerThreads;
- }
-
- @Managed
- public int getMinimumNumberOfDrivers()
- {
- return minimumNumberOfDrivers;
- }
-
- @Managed
- public synchronized int getTotalSplits()
- {
- return allSplits.size();
- }
-
- @Managed
- public synchronized int getIntermediateSplits()
- {
- return intermediateSplits.size();
- }
-
- @Managed
- public int getWaitingSplits()
- {
- return waitingSplits.size();
- }
-
- @Managed
- @Nested
- public DistributionStat getLeafSplitsSize()
- {
- return leafSplitsSize;
- }
-
- @Managed
- public int getRunningSplits()
- {
- return runningSplits.size();
- }
-
- @Managed
- public int getBlockedSplits()
- {
- return blockedSplits.size();
- }
-
- @Managed
- public long getCompletedTasksLevel0()
- {
- return completedTasksPerLevel.get(0);
- }
-
- @Managed
- public long getCompletedTasksLevel1()
- {
- return completedTasksPerLevel.get(1);
- }
-
- @Managed
- public long getCompletedTasksLevel2()
- {
- return completedTasksPerLevel.get(2);
- }
-
- @Managed
- public long getCompletedTasksLevel3()
- {
- return completedTasksPerLevel.get(3);
- }
-
- @Managed
- public long getCompletedTasksLevel4()
- {
- return completedTasksPerLevel.get(4);
- }
-
- @Managed
- public long getCompletedSplitsLevel0()
- {
- return completedSplitsPerLevel.get(0);
- }
-
- @Managed
- public long getCompletedSplitsLevel1()
- {
- return completedSplitsPerLevel.get(1);
- }
-
- @Managed
- public long getCompletedSplitsLevel2()
- {
- return completedSplitsPerLevel.get(2);
- }
-
- @Managed
- public long getCompletedSplitsLevel3()
- {
- return completedSplitsPerLevel.get(3);
- }
-
- @Managed
- public long getCompletedSplitsLevel4()
- {
- return completedSplitsPerLevel.get(4);
- }
-
- @Managed
- public long getRunningTasksLevel0()
- {
- return getRunningTasksForLevel(0);
- }
-
- @Managed
- public long getRunningTasksLevel1()
- {
- return getRunningTasksForLevel(1);
- }
-
- @Managed
- public long getRunningTasksLevel2()
- {
- return getRunningTasksForLevel(2);
- }
-
- @Managed
- public long getRunningTasksLevel3()
- {
- return getRunningTasksForLevel(3);
- }
-
- @Managed
- public long getRunningTasksLevel4()
- {
- return getRunningTasksForLevel(4);
- }
-
- @Managed
- @Nested
- public TimeStat getSplitQueuedTime()
- {
- return splitQueuedTime;
- }
-
- @Managed
- @Nested
- public TimeStat getSplitWallTime()
- {
- return splitWallTime;
- }
-
- @Managed
- @Nested
- public TimeStat getBlockedQuantaWallTime()
- {
- return blockedQuantaWallTime;
- }
-
- @Managed
- @Nested
- public TimeStat getUnblockedQuantaWallTime()
- {
- return unblockedQuantaWallTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getLeafSplitScheduledTime()
- {
- return leafSplitScheduledTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getIntermediateSplitScheduledTime()
- {
- return intermediateSplitScheduledTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getLeafSplitWallTime()
- {
- return leafSplitWallTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getIntermediateSplitWallTime()
- {
- return intermediateSplitWallTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getLeafSplitWaitTime()
- {
- return leafSplitWaitTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getIntermediateSplitWaitTime()
- {
- return intermediateSplitWaitTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getLeafSplitCpuTime()
- {
- return leafSplitCpuTime;
- }
-
- @Managed
- @Nested
- public TimeDistribution getIntermediateSplitCpuTime()
- {
- return intermediateSplitCpuTime;
- }
-
- @Managed
- @Nested
- public CounterStat getGlobalScheduledTimeMicros()
- {
- return globalScheduledTimeMicros;
- }
-
- @Managed
- @Nested
- public CounterStat getGlobalCpuTimeMicros()
- {
- return globalCpuTimeMicros;
- }
-
- private synchronized int getRunningTasksForLevel(int level)
- {
- int count = 0;
- for (TaskHandle task : tasks) {
- if (task.getPriority().getLevel() == level) {
- count++;
- }
- }
- return count;
- }
-
- public String getMaxActiveSplitsInfo()
- {
- // Sample output:
- //
- // 2 splits have been continuously active for more than 600.00ms seconds
- //
- // "20180907_054754_00000_88xi4.1.0-2" tid=99
- // at java.util.Formatter$FormatSpecifier.(Formatter.java:2708)
- // at java.util.Formatter.parse(Formatter.java:2560)
- // at java.util.Formatter.format(Formatter.java:2501)
- // at ... (more lines of stacktrace)
- //
- // "20180907_054754_00000_88xi4.1.0-3" tid=106
- // at java.util.Formatter$FormatSpecifier.(Formatter.java:2709)
- // at java.util.Formatter.parse(Formatter.java:2560)
- // at java.util.Formatter.format(Formatter.java:2501)
- // at ... (more line of stacktrace)
- StringBuilder stackTrace = new StringBuilder();
- int maxActiveSplitCount = 0;
- String message = "%s splits have been continuously active for more than %s seconds\n";
- for (RunningSplitInfo splitInfo : runningSplitInfos) {
- Duration duration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
- if (duration.compareTo(stuckSplitsWarningThreshold) >= 0) {
- maxActiveSplitCount++;
- stackTrace.append("\n");
- stackTrace.append(format("\"%s\" tid=%s", splitInfo.getThreadId(), splitInfo.getThread().getId())).append("\n");
- for (StackTraceElement traceElement : splitInfo.getThread().getStackTrace()) {
- stackTrace.append("\tat ").append(traceElement).append("\n");
- }
- }
- }
-
- return format(message, maxActiveSplitCount, stuckSplitsWarningThreshold).concat(stackTrace.toString());
- }
-
- @Managed
- public long getRunAwaySplitCount()
- {
- int count = 0;
- for (RunningSplitInfo splitInfo : runningSplitInfos) {
- Duration duration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
- if (duration.compareTo(stuckSplitsWarningThreshold) > 0) {
- count++;
- }
- }
- return count;
- }
-
- public Set getStuckSplitTaskIds(Duration processingDurationThreshold, Predicate filter)
- {
- return runningSplitInfos.stream()
- .filter((RunningSplitInfo splitInfo) -> {
- Duration splitProcessingDuration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
- return splitProcessingDuration.compareTo(processingDurationThreshold) > 0;
- })
- .filter(filter).map(RunningSplitInfo::getTaskId).collect(toImmutableSet());
- }
-
- /**
- * A class representing a split that is running on the TaskRunner.
- * It has a Thread object that gets assigned while assigning the split
- * to the taskRunner. However, when the TaskRunner moves to a different split,
- * the thread stored here will not remain assigned to this split anymore.
- */
- public static class RunningSplitInfo
- implements Comparable
- {
- private final long startTime;
- private final String threadId;
- private final Thread thread;
- private boolean printed;
- private final PrioritizedSplitRunner split;
-
- public RunningSplitInfo(long startTime, String threadId, Thread thread, PrioritizedSplitRunner split)
- {
- this.startTime = startTime;
- this.threadId = requireNonNull(threadId, "threadId is null");
- this.thread = requireNonNull(thread, "thread is null");
- this.split = requireNonNull(split, "split is null");
- this.printed = false;
- }
-
- public long getStartTime()
- {
- return startTime;
- }
-
- public String getThreadId()
- {
- return threadId;
- }
-
- public Thread getThread()
- {
- return thread;
- }
-
- public TaskId getTaskId()
- {
- return split.getTaskHandle().getTaskId();
- }
+ OptionalInt maxDriversPerTask);
- /**
- * {@link PrioritizedSplitRunner#getInfo()} provides runtime statistics for the split (such as total cpu utilization so far).
- * A value returned from this method changes over time and cannot be cached as a field of {@link RunningSplitInfo}.
- *
- * @return Formatted string containing runtime statistics for the split.
- */
- public String getSplitInfo()
- {
- return split.getInfo();
- }
+ void removeTask(TaskHandle taskHandle);
- public boolean isPrinted()
- {
- return printed;
- }
+ List> enqueueSplits(TaskHandle taskHandle, boolean intermediate, List extends SplitRunner> taskSplits);
- public void setPrinted()
- {
- printed = true;
- }
+ Set getStuckSplitTaskIds(Duration processingDurationThreshold, Predicate filter);
- @Override
- public int compareTo(RunningSplitInfo o)
- {
- return ComparisonChain.start()
- .compare(startTime, o.getStartTime())
- .compare(threadId, o.getThreadId())
- .result();
- }
- }
+ void start();
- @Managed(description = "Task processor executor")
- @Nested
- public ThreadPoolExecutorMBean getProcessorExecutor()
- {
- return executorMBean;
- }
+ void stop();
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/TaskHandle.java b/core/trino-main/src/main/java/io/trino/execution/executor/TaskHandle.java
index f3d99d0fb1b7..2a032768feea 100644
--- a/core/trino-main/src/main/java/io/trino/execution/executor/TaskHandle.java
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/TaskHandle.java
@@ -13,188 +13,7 @@
*/
package io.trino.execution.executor;
-import com.google.common.collect.ImmutableList;
-import com.google.errorprone.annotations.ThreadSafe;
-import com.google.errorprone.annotations.concurrent.GuardedBy;
-import io.airlift.units.Duration;
-import io.trino.execution.SplitConcurrencyController;
-import io.trino.execution.TaskId;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.OptionalInt;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.DoubleSupplier;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static java.util.Objects.requireNonNull;
-
-@ThreadSafe
-public class TaskHandle
+public interface TaskHandle
{
- private volatile boolean destroyed;
- private final TaskId taskId;
- private final DoubleSupplier utilizationSupplier;
-
- @GuardedBy("this")
- protected final Queue queuedLeafSplits = new ArrayDeque<>(10);
- @GuardedBy("this")
- protected final List runningLeafSplits = new ArrayList<>(10);
- @GuardedBy("this")
- protected final List runningIntermediateSplits = new ArrayList<>(10);
- @GuardedBy("this")
- protected long scheduledNanos;
- @GuardedBy("this")
- protected final SplitConcurrencyController concurrencyController;
-
- private final AtomicInteger nextSplitId = new AtomicInteger();
-
- private final AtomicReference priority = new AtomicReference<>(new Priority(0, 0));
- private final MultilevelSplitQueue splitQueue;
- private final OptionalInt maxDriversPerTask;
-
- public TaskHandle(
- TaskId taskId,
- MultilevelSplitQueue splitQueue,
- DoubleSupplier utilizationSupplier,
- int initialSplitConcurrency,
- Duration splitConcurrencyAdjustFrequency,
- OptionalInt maxDriversPerTask)
- {
- this.taskId = requireNonNull(taskId, "taskId is null");
- this.splitQueue = requireNonNull(splitQueue, "splitQueue is null");
- this.utilizationSupplier = requireNonNull(utilizationSupplier, "utilizationSupplier is null");
- this.maxDriversPerTask = requireNonNull(maxDriversPerTask, "maxDriversPerTask is null");
- this.concurrencyController = new SplitConcurrencyController(
- initialSplitConcurrency,
- requireNonNull(splitConcurrencyAdjustFrequency, "splitConcurrencyAdjustFrequency is null"));
- }
-
- public synchronized Priority addScheduledNanos(long durationNanos)
- {
- concurrencyController.update(durationNanos, utilizationSupplier.getAsDouble(), runningLeafSplits.size());
- scheduledNanos += durationNanos;
-
- Priority newPriority = splitQueue.updatePriority(priority.get(), durationNanos, scheduledNanos);
-
- priority.set(newPriority);
- return newPriority;
- }
-
- public synchronized Priority resetLevelPriority()
- {
- Priority currentPriority = priority.get();
- long levelMinPriority = splitQueue.getLevelMinPriority(currentPriority.getLevel(), scheduledNanos);
-
- if (currentPriority.getLevelPriority() < levelMinPriority) {
- Priority newPriority = new Priority(currentPriority.getLevel(), levelMinPriority);
- priority.set(newPriority);
- return newPriority;
- }
-
- return currentPriority;
- }
-
- public boolean isDestroyed()
- {
- return destroyed;
- }
-
- public Priority getPriority()
- {
- return priority.get();
- }
-
- public TaskId getTaskId()
- {
- return taskId;
- }
-
- public OptionalInt getMaxDriversPerTask()
- {
- return maxDriversPerTask;
- }
-
- // Returns any remaining splits. The caller must destroy these.
- public synchronized List destroy()
- {
- destroyed = true;
-
- ImmutableList.Builder builder = ImmutableList.builderWithExpectedSize(runningIntermediateSplits.size() + runningLeafSplits.size() + queuedLeafSplits.size());
- builder.addAll(runningIntermediateSplits);
- builder.addAll(runningLeafSplits);
- builder.addAll(queuedLeafSplits);
- runningIntermediateSplits.clear();
- runningLeafSplits.clear();
- queuedLeafSplits.clear();
- return builder.build();
- }
-
- public synchronized boolean enqueueSplit(PrioritizedSplitRunner split)
- {
- if (destroyed) {
- return false;
- }
- queuedLeafSplits.add(split);
- return true;
- }
-
- public synchronized boolean recordIntermediateSplit(PrioritizedSplitRunner split)
- {
- if (destroyed) {
- return false;
- }
- runningIntermediateSplits.add(split);
- return true;
- }
-
- synchronized int getRunningLeafSplits()
- {
- return runningLeafSplits.size();
- }
-
- public synchronized long getScheduledNanos()
- {
- return scheduledNanos;
- }
-
- public synchronized PrioritizedSplitRunner pollNextSplit()
- {
- if (destroyed) {
- return null;
- }
-
- if (runningLeafSplits.size() >= concurrencyController.getTargetConcurrency()) {
- return null;
- }
-
- PrioritizedSplitRunner split = queuedLeafSplits.poll();
- if (split != null) {
- runningLeafSplits.add(split);
- }
- return split;
- }
-
- public synchronized void splitComplete(PrioritizedSplitRunner split)
- {
- concurrencyController.splitFinished(split.getScheduledNanos(), utilizationSupplier.getAsDouble(), runningLeafSplits.size());
- runningIntermediateSplits.remove(split);
- runningLeafSplits.remove(split);
- }
-
- public int getNextSplitId()
- {
- return nextSplitId.getAndIncrement();
- }
-
- @Override
- public String toString()
- {
- return toStringHelper(this)
- .add("taskId", taskId)
- .toString();
- }
+ boolean isDestroyed();
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/SplitProcessor.java b/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/SplitProcessor.java
new file mode 100644
index 000000000000..87efbed973a0
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/SplitProcessor.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.dedicated;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.concurrent.SetThreadName;
+import io.airlift.log.Logger;
+import io.airlift.stats.CpuTimer;
+import io.airlift.units.Duration;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.SpanBuilder;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.context.Context;
+import io.trino.execution.SplitRunner;
+import io.trino.execution.TaskId;
+import io.trino.execution.executor.scheduler.Schedulable;
+import io.trino.execution.executor.scheduler.SchedulerContext;
+import io.trino.tracing.TrinoAttributes;
+
+import java.util.concurrent.TimeUnit;
+
+import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+class SplitProcessor
+ implements Schedulable
+{
+ private static final Logger LOG = Logger.get(SplitProcessor.class);
+
+ private static final Duration SPLIT_RUN_QUANTA = new Duration(1, TimeUnit.SECONDS);
+
+ private final TaskId taskId;
+ private final int splitId;
+ private final SplitRunner split;
+ private final Tracer tracer;
+
+ public SplitProcessor(TaskId taskId, int splitId, SplitRunner split, Tracer tracer)
+ {
+ this.taskId = requireNonNull(taskId, "taskId is null");
+ this.splitId = splitId;
+ this.split = requireNonNull(split, "split is null");
+ this.tracer = requireNonNull(tracer, "tracer is null");
+ }
+
+ @Override
+ public void run(SchedulerContext context)
+ {
+ Span splitSpan = tracer.spanBuilder("split")
+ .setParent(Context.current().with(split.getPipelineSpan()))
+ .setAttribute(TrinoAttributes.QUERY_ID, taskId.getQueryId().toString())
+ .setAttribute(TrinoAttributes.STAGE_ID, taskId.getStageId().toString())
+ .setAttribute(TrinoAttributes.TASK_ID, taskId.toString())
+ .setAttribute(TrinoAttributes.PIPELINE_ID, taskId.getStageId() + "-" + split.getPipelineId())
+ .setAttribute(TrinoAttributes.SPLIT_ID, taskId + "-" + splitId)
+ .startSpan();
+
+ Span processSpan = newSpan(splitSpan, null);
+
+ CpuTimer timer = new CpuTimer(Ticker.systemTicker(), false);
+ long previousCpuNanos = 0;
+ long previousScheduledNanos = 0;
+ try (SetThreadName ignored = new SetThreadName("SplitRunner-%s-%s", taskId, splitId)) {
+ while (!split.isFinished()) {
+ ListenableFuture blocked = split.processFor(SPLIT_RUN_QUANTA);
+ CpuTimer.CpuDuration elapsed = timer.elapsedTime();
+
+ long scheduledNanos = elapsed.getWall().roundTo(NANOSECONDS);
+ processSpan.setAttribute(TrinoAttributes.SPLIT_SCHEDULED_TIME_NANOS, scheduledNanos - previousScheduledNanos);
+ previousScheduledNanos = scheduledNanos;
+
+ long cpuNanos = elapsed.getCpu().roundTo(NANOSECONDS);
+ processSpan.setAttribute(TrinoAttributes.SPLIT_CPU_TIME_NANOS, cpuNanos - previousCpuNanos);
+ previousCpuNanos = cpuNanos;
+
+ if (!split.isFinished()) {
+ if (blocked.isDone()) {
+ processSpan.addEvent("yield");
+ processSpan.end();
+ if (!context.maybeYield()) {
+ processSpan = null;
+ return;
+ }
+ }
+ else {
+ processSpan.addEvent("blocked");
+ processSpan.end();
+ if (!context.block(blocked)) {
+ processSpan = null;
+ return;
+ }
+ }
+ processSpan = newSpan(splitSpan, processSpan);
+ }
+ }
+ }
+ catch (Exception e) {
+ LOG.error(e);
+ }
+ finally {
+ if (processSpan != null) {
+ processSpan.end();
+ }
+
+ splitSpan.setAttribute(TrinoAttributes.SPLIT_CPU_TIME_NANOS, timer.elapsedTime().getCpu().roundTo(NANOSECONDS));
+ splitSpan.setAttribute(TrinoAttributes.SPLIT_SCHEDULED_TIME_NANOS, context.getScheduledNanos());
+ splitSpan.setAttribute(TrinoAttributes.SPLIT_BLOCK_TIME_NANOS, context.getBlockedNanos());
+ splitSpan.setAttribute(TrinoAttributes.SPLIT_WAIT_TIME_NANOS, context.getWaitNanos());
+ splitSpan.setAttribute(TrinoAttributes.SPLIT_START_TIME_NANOS, context.getStartNanos());
+ splitSpan.end();
+ }
+ }
+
+ private Span newSpan(Span parent, Span previous)
+ {
+ SpanBuilder builder = tracer.spanBuilder("process")
+ .setParent(Context.current().with(parent));
+
+ if (previous != null) {
+ builder.addLink(previous.getSpanContext());
+ }
+
+ return builder.startSpan();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/ThreadPerDriverTaskExecutor.java b/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/ThreadPerDriverTaskExecutor.java
new file mode 100644
index 000000000000..4a2a42ad39ae
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/dedicated/ThreadPerDriverTaskExecutor.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.dedicated;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Ticker;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import com.google.inject.Inject;
+import io.airlift.units.Duration;
+import io.opentelemetry.api.trace.Tracer;
+import io.trino.execution.SplitRunner;
+import io.trino.execution.TaskId;
+import io.trino.execution.TaskManagerConfig;
+import io.trino.execution.executor.RunningSplitInfo;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.TaskHandle;
+import io.trino.execution.executor.scheduler.FairScheduler;
+import io.trino.execution.executor.scheduler.Group;
+import io.trino.execution.executor.scheduler.Schedulable;
+import io.trino.execution.executor.scheduler.SchedulerContext;
+import io.trino.spi.VersionEmbedder;
+import jakarta.annotation.PostConstruct;
+import jakarta.annotation.PreDestroy;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.OptionalInt;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.DoubleSupplier;
+import java.util.function.Predicate;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
+import static java.util.Objects.requireNonNull;
+
+@ThreadSafe
+public class ThreadPerDriverTaskExecutor
+ implements TaskExecutor
+{
+ private final FairScheduler scheduler;
+ private final Tracer tracer;
+ private final VersionEmbedder versionEmbedder;
+ private volatile boolean closed;
+
+ @Inject
+ public ThreadPerDriverTaskExecutor(TaskManagerConfig config, Tracer tracer, VersionEmbedder versionEmbedder)
+ {
+ this(tracer, versionEmbedder, new FairScheduler(config.getMaxWorkerThreads(), "SplitRunner-%d", Ticker.systemTicker()));
+ }
+
+ @VisibleForTesting
+ public ThreadPerDriverTaskExecutor(Tracer tracer, VersionEmbedder versionEmbedder, FairScheduler scheduler)
+ {
+ this.scheduler = scheduler;
+ this.tracer = requireNonNull(tracer, "tracer is null");
+ this.versionEmbedder = requireNonNull(versionEmbedder, "versionEmbedder is null");
+ }
+
+ @PostConstruct
+ @Override
+ public synchronized void start()
+ {
+ scheduler.start();
+ }
+
+ @PreDestroy
+ @Override
+ public synchronized void stop()
+ {
+ closed = true;
+ scheduler.close();
+ }
+
+ @Override
+ public synchronized TaskHandle addTask(
+ TaskId taskId,
+ DoubleSupplier utilizationSupplier,
+ int initialSplitConcurrency,
+ Duration splitConcurrencyAdjustFrequency,
+ OptionalInt maxDriversPerTask)
+ {
+ checkArgument(!closed, "Executor is already closed");
+
+ Group group = scheduler.createGroup(taskId.toString());
+ return new TaskEntry(taskId, group);
+ }
+
+ @Override
+ public synchronized void removeTask(TaskHandle handle)
+ {
+ TaskEntry entry = (TaskEntry) handle;
+
+ if (!entry.isDestroyed()) {
+ scheduler.removeGroup(entry.group());
+ entry.destroy();
+ }
+ }
+
+ @Override
+ public synchronized List> enqueueSplits(TaskHandle handle, boolean intermediate, List extends SplitRunner> splits)
+ {
+ checkArgument(!closed, "Executor is already closed");
+
+ TaskEntry entry = (TaskEntry) handle;
+
+ List> futures = new ArrayList<>();
+ for (SplitRunner split : splits) {
+ entry.addSplit(split);
+
+ int splitId = entry.nextSplitId();
+ ListenableFuture done = scheduler.submit(entry.group(), splitId, new VersionEmbedderBridge(versionEmbedder, new SplitProcessor(entry.taskId(), splitId, split, tracer)));
+ done.addListener(split::close, directExecutor());
+ futures.add(done);
+ }
+
+ return futures;
+ }
+
+ @Override
+ public Set getStuckSplitTaskIds(Duration processingDurationThreshold, Predicate filter)
+ {
+ // TODO
+ return ImmutableSet.of();
+ }
+
+ private static class TaskEntry
+ implements TaskHandle
+ {
+ private final TaskId taskId;
+ private final Group group;
+ private final AtomicInteger nextSplitId = new AtomicInteger();
+ private volatile boolean destroyed;
+
+ @GuardedBy("this")
+ private Set splits = new HashSet<>();
+
+ public TaskEntry(TaskId taskId, Group group)
+ {
+ this.taskId = taskId;
+ this.group = group;
+ }
+
+ public TaskId taskId()
+ {
+ return taskId;
+ }
+
+ public Group group()
+ {
+ return group;
+ }
+
+ public synchronized void destroy()
+ {
+ destroyed = true;
+
+ for (SplitRunner split : splits) {
+ split.close();
+ }
+ }
+
+ public synchronized void addSplit(SplitRunner split)
+ {
+ checkArgument(!destroyed, "Task already destroyed: %s", taskId);
+ splits.add(split);
+ }
+
+ public int nextSplitId()
+ {
+ return nextSplitId.incrementAndGet();
+ }
+
+ @Override
+ public boolean isDestroyed()
+ {
+ return destroyed;
+ }
+ }
+
+ private record VersionEmbedderBridge(VersionEmbedder versionEmbedder, Schedulable delegate)
+ implements Schedulable
+ {
+ @Override
+ public void run(SchedulerContext context)
+ {
+ Runnable adapter = () -> delegate.run(context);
+ versionEmbedder.embedVersion(adapter).run();
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/BlockingSchedulingQueue.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/BlockingSchedulingQueue.java
new file mode 100644
index 000000000000..093017772fa1
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/BlockingSchedulingQueue.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+
+import java.util.Set;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+@ThreadSafe
+final class BlockingSchedulingQueue
+{
+ private final Lock lock = new ReentrantLock();
+ private final Condition notEmpty = lock.newCondition();
+
+ @GuardedBy("lock")
+ private final SchedulingQueue queue = new SchedulingQueue<>();
+
+ public void startGroup(G group)
+ {
+ lock.lock();
+ try {
+ queue.startGroup(group);
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public Set finishGroup(G group)
+ {
+ lock.lock();
+ try {
+ return queue.finishGroup(group);
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public Set finishAll()
+ {
+ lock.lock();
+ try {
+ return queue.finishAll();
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public boolean enqueue(G group, T task, long deltaWeight)
+ {
+ lock.lock();
+ try {
+ if (!queue.containsGroup(group)) {
+ return false;
+ }
+
+ queue.enqueue(group, task, deltaWeight);
+ notEmpty.signal();
+
+ return true;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public boolean block(G group, T task, long deltaWeight)
+ {
+ lock.lock();
+ try {
+ if (!queue.containsGroup(group)) {
+ return false;
+ }
+
+ queue.block(group, task, deltaWeight);
+ return true;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public T dequeue(long expectedWeight)
+ throws InterruptedException
+ {
+ lock.lock();
+ try {
+ T result;
+ do {
+ result = queue.dequeue(expectedWeight);
+ if (result == null) {
+ notEmpty.await();
+ }
+ }
+ while (result == null);
+
+ return result;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ @Override
+ public String toString()
+ {
+ lock.lock();
+ try {
+ return queue.toString();
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public int getRunnableCount()
+ {
+ lock.lock();
+ try {
+ return queue.getRunnableCount();
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/FairScheduler.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/FairScheduler.java
new file mode 100644
index 000000000000..6c5f82dec7d3
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/FairScheduler.java
@@ -0,0 +1,306 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import io.airlift.log.Logger;
+
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Implementation nodes
+ *
+ *
+ * - The TaskControl state machine is only modified by the task executor
+ * thread (i.e., from within {@link FairScheduler#runTask(Schedulable, TaskControl)} )}). Other threads
+ * can indirectly affect what the task executor thread does by marking the task as ready or cancelled
+ * and unblocking the task executor thread, which will then act on that information.
+ *
+ */
+@ThreadSafe
+public final class FairScheduler
+ implements AutoCloseable
+{
+ private static final Logger LOG = Logger.get(FairScheduler.class);
+
+ public static final long QUANTUM_NANOS = TimeUnit.MILLISECONDS.toNanos(1000);
+
+ private final ExecutorService schedulerExecutor;
+ private final ListeningExecutorService taskExecutor;
+ private final BlockingSchedulingQueue queue = new BlockingSchedulingQueue<>();
+ private final Reservation concurrencyControl;
+ private final Ticker ticker;
+
+ private final Gate paused = new Gate(true);
+
+ @GuardedBy("this")
+ private boolean closed;
+
+ public FairScheduler(int maxConcurrentTasks, String threadNameFormat, Ticker ticker)
+ {
+ this.ticker = requireNonNull(ticker, "ticker is null");
+
+ concurrencyControl = new Reservation<>(maxConcurrentTasks);
+
+ schedulerExecutor = Executors.newCachedThreadPool(new ThreadFactoryBuilder()
+ .setNameFormat("fair-scheduler-%d")
+ .setDaemon(true)
+ .build());
+
+ taskExecutor = MoreExecutors.listeningDecorator(Executors.newCachedThreadPool(new ThreadFactoryBuilder()
+ .setNameFormat(threadNameFormat)
+ .setDaemon(true)
+ .build()));
+ }
+
+ public static FairScheduler newInstance(int maxConcurrentTasks)
+ {
+ return newInstance(maxConcurrentTasks, Ticker.systemTicker());
+ }
+
+ public static FairScheduler newInstance(int maxConcurrentTasks, Ticker ticker)
+ {
+ FairScheduler scheduler = new FairScheduler(maxConcurrentTasks, "fair-scheduler-runner-%d", ticker);
+ scheduler.start();
+ return scheduler;
+ }
+
+ public void start()
+ {
+ schedulerExecutor.submit(this::runScheduler);
+ }
+
+ public void pause()
+ {
+ paused.close();
+ }
+
+ public void resume()
+ {
+ paused.open();
+ }
+
+ @Override
+ public synchronized void close()
+ {
+ if (closed) {
+ return;
+ }
+ closed = true;
+
+ Set tasks = queue.finishAll();
+
+ for (TaskControl task : tasks) {
+ task.cancel();
+ }
+
+ taskExecutor.shutdownNow();
+ schedulerExecutor.shutdownNow();
+ }
+
+ public synchronized Group createGroup(String name)
+ {
+ checkArgument(!closed, "Already closed");
+
+ Group group = new Group(name);
+ queue.startGroup(group);
+
+ return group;
+ }
+
+ public synchronized void removeGroup(Group group)
+ {
+ checkArgument(!closed, "Already closed");
+
+ Set tasks = queue.finishGroup(group);
+
+ for (TaskControl task : tasks) {
+ task.cancel();
+ }
+ }
+
+ public synchronized ListenableFuture submit(Group group, int id, Schedulable runner)
+ {
+ checkArgument(!closed, "Already closed");
+
+ TaskControl task = new TaskControl(group, id, ticker);
+
+ return taskExecutor.submit(() -> runTask(runner, task), null);
+ }
+
+ private void runTask(Schedulable runner, TaskControl task)
+ {
+ task.setThread(Thread.currentThread());
+
+ if (!makeRunnableAndAwait(task, 0)) {
+ return;
+ }
+
+ SchedulerContext context = new SchedulerContext(this, task);
+ try {
+ runner.run(context);
+ }
+ catch (Exception e) {
+ LOG.error(e);
+ }
+ finally {
+ // If the runner exited due to an exception in user code or
+ // normally (not in response to an interruption during blocking or yield),
+ // it must have had a semaphore permit reserved, so release it.
+ if (task.getState() == TaskControl.State.RUNNING) {
+ concurrencyControl.release(task);
+ }
+ task.transitionToFinished();
+ }
+ }
+
+ private boolean makeRunnableAndAwait(TaskControl task, long deltaWeight)
+ {
+ if (!task.transitionToWaiting()) {
+ return false;
+ }
+
+ if (!queue.enqueue(task.group(), task, deltaWeight)) {
+ return false;
+ }
+
+ // wait for the task to be scheduled
+ return awaitReadyAndTransitionToRunning(task);
+ }
+
+ /**
+ * @return false if the transition was unsuccessful due to the task being cancelled
+ */
+ private boolean awaitReadyAndTransitionToRunning(TaskControl task)
+ {
+ if (!task.awaitReady()) {
+ if (task.isReady()) {
+ // If the task was marked as ready (slot acquired) but then cancelled before
+ // awaitReady() was notified, we need to release the slot.
+ concurrencyControl.release(task);
+ }
+ return false;
+ }
+
+ if (!task.transitionToRunning()) {
+ concurrencyControl.release(task);
+ return false;
+ }
+
+ return true;
+ }
+
+ boolean yield(TaskControl task)
+ {
+ checkState(task.getThread() == Thread.currentThread(), "yield() may only be called from the task thread");
+
+ long delta = task.elapsed();
+ if (delta < QUANTUM_NANOS) {
+ return true;
+ }
+
+ concurrencyControl.release(task);
+
+ return makeRunnableAndAwait(task, delta);
+ }
+
+ boolean block(TaskControl task, ListenableFuture> future)
+ {
+ checkState(task.getThread() == Thread.currentThread(), "block() may only be called from the task thread");
+
+ long delta = task.elapsed();
+
+ concurrencyControl.release(task);
+
+ if (!task.transitionToBlocked()) {
+ return false;
+ }
+
+ if (!queue.block(task.group(), task, delta)) {
+ return false;
+ }
+
+ future.addListener(task::markUnblocked, MoreExecutors.directExecutor());
+ task.awaitUnblock();
+
+ return makeRunnableAndAwait(task, 0);
+ }
+
+ private void runScheduler()
+ {
+ while (true) {
+ try {
+ paused.awaitOpen();
+ concurrencyControl.reserve();
+ TaskControl task = queue.dequeue(QUANTUM_NANOS);
+
+ concurrencyControl.register(task);
+ if (!task.markReady()) {
+ concurrencyControl.release(task);
+ }
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return;
+ }
+ catch (Exception e) {
+ LOG.error(e);
+ }
+ }
+ }
+
+ long getStartNanos(TaskControl task)
+ {
+ return task.getStartNanos();
+ }
+
+ long getScheduledNanos(TaskControl task)
+ {
+ return task.getScheduledNanos();
+ }
+
+ long getWaitNanos(TaskControl task)
+ {
+ return task.getWaitNanos();
+ }
+
+ long getBlockedNanos(TaskControl task)
+ {
+ return task.getBlockedNanos();
+ }
+
+ @Override
+ public String toString()
+ {
+ return new StringJoiner(", ", FairScheduler.class.getSimpleName() + "[", "]")
+ .add("queue=" + queue)
+ .add("concurrencyControl=" + concurrencyControl)
+ .add("closed=" + closed)
+ .toString();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Gate.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Gate.java
new file mode 100644
index 000000000000..adcc930b6f24
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Gate.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.errorprone.annotations.ThreadSafe;
+
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+@ThreadSafe
+final class Gate
+{
+ private final Lock lock = new ReentrantLock();
+ private final Condition opened = lock.newCondition();
+ private boolean open;
+
+ public Gate(boolean opened)
+ {
+ this.open = opened;
+ }
+
+ public void close()
+ {
+ lock.lock();
+ try {
+ open = false;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public void open()
+ {
+ lock.lock();
+ try {
+ open = true;
+ opened.signalAll();
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public void awaitOpen()
+ throws InterruptedException
+ {
+ lock.lock();
+ try {
+ while (!open) {
+ opened.await();
+ }
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Group.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Group.java
new file mode 100644
index 000000000000..596abffbe1e3
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Group.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+public record Group(String name, long startTime)
+{
+ public Group(String name)
+ {
+ this(name, System.nanoTime());
+ }
+
+ @Override
+ public String toString()
+ {
+ return name;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/PriorityQueue.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/PriorityQueue.java
new file mode 100644
index 000000000000..7724400f4c50
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/PriorityQueue.java
@@ -0,0 +1,149 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import io.trino.annotation.NotThreadSafe;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+
+@NotThreadSafe
+final class PriorityQueue
+{
+ // The tree is ordered by priorities in this map, so any operations on the data
+ // structures needs to consider the importance of the relative order of the operations.
+ // For instance, removing an entry from the tree before the corresponding entry in the
+ // queue is removed will lead to NPEs.
+ private final Map priorities = new HashMap<>();
+ private final TreeSet queue;
+
+ private long sequence;
+
+ public PriorityQueue()
+ {
+ queue = new TreeSet<>((a, b) -> {
+ Priority first = priorities.get(a);
+ Priority second = priorities.get(b);
+
+ int result = Long.compare(first.priority(), second.priority());
+ if (result == 0) {
+ result = Long.compare(first.sequence(), second.sequence());
+ }
+ return result;
+ });
+ }
+
+ public void add(T value, long priority)
+ {
+ checkArgument(!priorities.containsKey(value), "Value already in queue: %s", value);
+ priorities.put(value, new Priority(priority, nextSequence()));
+ queue.add(value);
+ }
+
+ public void addOrReplace(T value, long priority)
+ {
+ if (priorities.containsKey(value)) {
+ queue.remove(value);
+ priorities.put(value, new Priority(priority, nextSequence()));
+ queue.add(value);
+ }
+ else {
+ add(value, priority);
+ }
+ }
+
+ public T takeOrThrow()
+ {
+ T result = poll();
+ checkState(result != null, "Queue is empty");
+ return result;
+ }
+
+ public T poll()
+ {
+ T result = queue.pollFirst();
+ if (result != null) {
+ priorities.remove(result);
+ }
+
+ return result;
+ }
+
+ public void remove(T value)
+ {
+ checkArgument(priorities.containsKey(value), "Value not in queue: %s", value);
+ queue.remove(value);
+ priorities.remove(value);
+ }
+
+ public void removeIfPresent(T value)
+ {
+ if (priorities.containsKey(value)) {
+ queue.remove(value);
+ priorities.remove(value);
+ }
+ }
+
+ public boolean contains(T value)
+ {
+ return priorities.containsKey(value);
+ }
+
+ public boolean isEmpty()
+ {
+ return priorities.isEmpty();
+ }
+
+ public Set values()
+ {
+ return priorities.keySet();
+ }
+
+ public long nextPriority()
+ {
+ checkState(!queue.isEmpty(), "Queue is empty");
+ return priorities.get(queue.first()).priority();
+ }
+
+ public T peek()
+ {
+ if (queue.isEmpty()) {
+ return null;
+ }
+ return queue.first();
+ }
+
+ public int size()
+ {
+ return queue.size();
+ }
+
+ @Override
+ public String toString()
+ {
+ return queue.toString();
+ }
+
+ private long nextSequence()
+ {
+ return sequence++;
+ }
+
+ private record Priority(long priority, long sequence) {}
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Reservation.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Reservation.java
new file mode 100644
index 000000000000..85d484c99d36
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Reservation.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.errorprone.annotations.ThreadSafe;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.StringJoiner;
+import java.util.concurrent.Semaphore;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Semaphore-like structure that allows for tracking reservations to avoid double-reserving or double-releasing.
+ *
+ * Callers are expected to call {@link #reserve()} to acquire a slot, and then {@link #register(T)} to associate
+ * an entity with the reservation.
+ *
+ * Upon completion, callers should call {@link #release(T)} to release the reservation.
+ */
+@ThreadSafe
+final class Reservation
+{
+ private final Semaphore semaphore;
+ private final Set reservations = new HashSet<>();
+
+ public Reservation(int slots)
+ {
+ semaphore = new Semaphore(slots);
+ }
+
+ public int availablePermits()
+ {
+ return semaphore.availablePermits();
+ }
+
+ public void reserve()
+ throws InterruptedException
+ {
+ semaphore.acquire();
+ }
+
+ public synchronized void register(T entry)
+ {
+ checkArgument(!reservations.contains(entry), "Already acquired: %s", entry);
+ reservations.add(entry);
+ }
+
+ public synchronized void release(T entry)
+ {
+ checkArgument(reservations.contains(entry), "Already released: %s", entry);
+ reservations.remove(entry);
+
+ semaphore.release();
+ }
+
+ public synchronized Set reservations()
+ {
+ return ImmutableSet.copyOf(reservations);
+ }
+
+ @Override
+ public synchronized String toString()
+ {
+ return new StringJoiner(", ", Reservation.class.getSimpleName() + "[", "]")
+ .add("semaphore=" + semaphore)
+ .add("reservations=" + reservations)
+ .toString();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Schedulable.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Schedulable.java
new file mode 100644
index 000000000000..39032d7dafff
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Schedulable.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+public interface Schedulable
+{
+ void run(SchedulerContext context);
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulerContext.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulerContext.java
new file mode 100644
index 000000000000..789086ff08e8
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulerContext.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import io.trino.annotation.NotThreadSafe;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@NotThreadSafe
+public final class SchedulerContext
+{
+ private final FairScheduler scheduler;
+ private final TaskControl handle;
+
+ public SchedulerContext(FairScheduler scheduler, TaskControl handle)
+ {
+ this.scheduler = scheduler;
+ this.handle = handle;
+ }
+
+ /**
+ * Attempt to relinquish control to let other tasks run.
+ *
+ * @return false if the task was interrupted or cancelled while yielding,
+ * for example if the Java thread was interrupted, the scheduler was shutdown,
+ * or the scheduling group was removed. The caller is expected to clean up and finish.
+ */
+ public boolean maybeYield()
+ {
+ checkArgument(handle.getState() == TaskControl.State.RUNNING, "Task is not running");
+
+ return scheduler.yield(handle);
+ }
+
+ /**
+ * Indicate that the current task is blocked. The method returns when the future
+ * completes of it the task is interrupted.
+ *
+ * @return false if the task was interrupted or cancelled while blocked,
+ * for example if the Java thread was interrupted, the scheduler was shutdown,
+ * or the scheduling group was removed. The caller is expected to clean up and finish.
+ */
+ public boolean block(ListenableFuture> future)
+ {
+ checkArgument(handle.getState() == TaskControl.State.RUNNING, "Task is not running");
+
+ return scheduler.block(handle, future);
+ }
+
+ public long getStartNanos()
+ {
+ return scheduler.getStartNanos(handle);
+ }
+
+ public long getWaitNanos()
+ {
+ return scheduler.getWaitNanos(handle);
+ }
+
+ public long getScheduledNanos()
+ {
+ return scheduler.getScheduledNanos(handle);
+ }
+
+ public long getBlockedNanos()
+ {
+ return scheduler.getBlockedNanos(handle);
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingGroup.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingGroup.java
new file mode 100644
index 000000000000..6eb20732e667
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingGroup.java
@@ -0,0 +1,194 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.collect.ImmutableSet;
+import io.trino.annotation.NotThreadSafe;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static io.trino.execution.executor.scheduler.State.BLOCKED;
+import static io.trino.execution.executor.scheduler.State.RUNNABLE;
+import static io.trino.execution.executor.scheduler.State.RUNNING;
+
+@NotThreadSafe
+final class SchedulingGroup
+{
+ private State state;
+ private long weight;
+ private final Map tasks = new HashMap<>();
+ private final PriorityQueue runnableQueue = new PriorityQueue<>();
+ private final Set blocked = new HashSet<>();
+ private final PriorityQueue baselineWeights = new PriorityQueue<>();
+
+ public SchedulingGroup()
+ {
+ this.state = BLOCKED;
+ }
+
+ public void enqueue(T handle, long deltaWeight)
+ {
+ Task task = tasks.get(handle);
+
+ if (task == null) {
+ // New tasks get assigned the baseline weight so that they don't monopolize the queue
+ // while they catch up
+ task = new Task(baselineWeight());
+ tasks.put(handle, task);
+ }
+ else if (task.state() == BLOCKED) {
+ blocked.remove(handle);
+ task.addWeight(baselineWeight());
+ }
+
+ weight -= task.uncommittedWeight();
+ weight += deltaWeight;
+
+ task.commitWeight(deltaWeight);
+ task.setState(RUNNABLE);
+ runnableQueue.add(handle, task.weight());
+ baselineWeights.addOrReplace(handle, task.weight());
+
+ updateState();
+ }
+
+ public T dequeue(long expectedWeight)
+ {
+ checkArgument(state == RUNNABLE);
+
+ T task = runnableQueue.takeOrThrow();
+
+ Task info = tasks.get(task);
+ info.setUncommittedWeight(expectedWeight);
+ info.setState(RUNNING);
+ weight += expectedWeight;
+
+ baselineWeights.addOrReplace(task, info.weight());
+
+ updateState();
+
+ return task;
+ }
+
+ public void finish(T task)
+ {
+ checkArgument(tasks.containsKey(task), "Unknown task: %s", task);
+ tasks.remove(task);
+ blocked.remove(task);
+ runnableQueue.removeIfPresent(task);
+ baselineWeights.removeIfPresent(task);
+
+ updateState();
+ }
+
+ public void block(T handle, long deltaWeight)
+ {
+ checkArgument(tasks.containsKey(handle), "Unknown task: %s", handle);
+ checkArgument(!runnableQueue.contains(handle), "Task is already in queue: %s", handle);
+
+ weight += deltaWeight;
+
+ Task task = tasks.get(handle);
+ task.commitWeight(deltaWeight);
+ task.setState(BLOCKED);
+ task.addWeight(-baselineWeight());
+ blocked.add(handle);
+ baselineWeights.remove(handle);
+
+ updateState();
+ }
+
+ public long baselineWeight()
+ {
+ if (baselineWeights.isEmpty()) {
+ return 0;
+ }
+
+ return baselineWeights.nextPriority();
+ }
+
+ public void addWeight(long delta)
+ {
+ weight += delta;
+ }
+
+ private void updateState()
+ {
+ if (blocked.size() == tasks.size()) {
+ state = BLOCKED;
+ }
+ else if (runnableQueue.isEmpty()) {
+ state = RUNNING;
+ }
+ else {
+ state = RUNNABLE;
+ }
+ }
+
+ public long weight()
+ {
+ return weight;
+ }
+
+ public Set tasks()
+ {
+ return ImmutableSet.copyOf(tasks.keySet());
+ }
+
+ public State state()
+ {
+ return state;
+ }
+
+ public T peek()
+ {
+ return runnableQueue.peek();
+ }
+
+ public int runnableCount()
+ {
+ return runnableQueue.size();
+ }
+
+ @Override
+ public String toString()
+ {
+ StringBuilder builder = new StringBuilder();
+ for (Map.Entry entry : tasks.entrySet()) {
+ T key = entry.getKey();
+ Task task = entry.getValue();
+
+ String prefix = "%s %s".formatted(
+ key == peek() ? "=>" : " ",
+ key);
+
+ String details = switch (task.state()) {
+ case BLOCKED -> "[BLOCKED, saved delta = %s]".formatted(task.weight());
+ case RUNNABLE -> "[RUNNABLE, weight = %s]".formatted(task.weight());
+ case RUNNING -> "[RUNNING, weight = %s, uncommitted = %s]".formatted(task.weight(), task.uncommittedWeight());
+ };
+
+ builder.append(prefix)
+ .append(" ")
+ .append(details)
+ .append("\n");
+ }
+
+ return builder.toString();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingQueue.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingQueue.java
new file mode 100644
index 000000000000..89a5278a5fd2
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/SchedulingQueue.java
@@ -0,0 +1,345 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.collect.ImmutableSet;
+import io.trino.annotation.NotThreadSafe;
+
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.base.Verify.verify;
+import static io.trino.execution.executor.scheduler.State.BLOCKED;
+import static io.trino.execution.executor.scheduler.State.RUNNABLE;
+import static io.trino.execution.executor.scheduler.State.RUNNING;
+
+/**
+ * A queue of tasks that are scheduled for execution. Modeled after
+ * Completely Fair Scheduler.
+ * Tasks are grouped into scheduling groups. Within a group, tasks are ordered based
+ * on their relative weight. Groups are ordered relative to each other based on the
+ * accumulated weight of their tasks.
+ *
+ * A task can be in one of three states:
+ *
+ * - runnable: the task is ready to run and waiting to be dequeued
+ *
- running: the task has been dequeued and is running
+ *
- blocked: the task is blocked on some external event and is not running
+ *
+ *
+ *
+ * A group can be in one of three states:
+ *
+ * - runnable: the group has at least one runnable task
+ *
- running: all the tasks in the group are currently running
+ *
- blocked: all the tasks in the group are currently blocked
+ *
+ *
+ *
+ * The goal is to balance the consideration among groups to ensure the accumulated
+ * weight in the long run is equal among groups. Within a group, the goal is to
+ * balance the consideration among tasks to ensure the accumulated weight in the
+ * long run is equal among tasks within the group.
+ *
+ *
Groups start in the blocked state and transition to the runnable state when a task is
+ * added via the {@link #enqueue(Object, Object, long)} method.
+ *
+ * Tasks are dequeued via the {@link #dequeue(long)}. When all tasks in a group have
+ * been dequeued, the group transitions to the running state and is removed from the
+ * queue.
+ *
+ * When a task time slice completes, it needs to be re-enqueued via the
+ * {@link #enqueue(Object, Object, long)}, which includes the desired
+ * increment in relative weight to apply to the task for further prioritization.
+ * The weight increment is also applied to the group.
+ *
+ *
+ * If a task blocks, the caller must call the {@link #block(Object, Object, long)}
+ * method to indicate that the task is no longer running. A weight increment can be
+ * included for the portion of time the task was not blocked.
+ *
+ * Group state transitions
+ *
+ * blockTask()
+ * finishTask() enqueueTask() enqueueTask()
+ * ┌───┐ ┌──────────────────────────────────────────┐ ┌────┐
+ * │ │ │ │ │ │
+ * │ ▼ │ ▼ ▼ │
+ * ┌─┴───────┴─┐ all blocked finishTask() ┌────────────┐ │
+ * │ │◄──────────────O◄────────────────────┤ ├──┘
+ * ────►│ BLOCKED │ │ │ RUNNABLE │
+ * │ │ │ ┌────────────────►│ │◄───┐
+ * └───────────┘ not all │ │ enqueueTask() └──────┬─────┘ │
+ * ▲ blocked │ │ │ │
+ * │ │ │ dequeueTask()│ │
+ * │ all blocked ▼ │ │ │
+ * │ ┌─────┴─────┐ ▼ │
+ * │ │ │◄─────────────────O──────────┘
+ * O◄──────────────────┤ RUNNING │ queue empty queue
+ * │ blockTask() │ ├───┐ not empty
+ * │ └───────────┘ │
+ * │ ▲ ▲ │ finishTask()
+ * └─────────────────────┘ └──────┘
+ * not all blocked
+ *
+ *
+ *
+ * Implementation notes
+ *
+ * - TODO: Initial weight upon registration
+ * - TODO: Weight adjustment during blocking / unblocking
+ * - TODO: Uncommitted weight on dequeue
+ *
+ *
+ */
+@NotThreadSafe
+final class SchedulingQueue
+{
+ private final PriorityQueue runnableQueue = new PriorityQueue<>();
+ private final Map> groups = new HashMap<>();
+ private final PriorityQueue baselineWeights = new PriorityQueue<>();
+
+ public void startGroup(G group)
+ {
+ checkArgument(!groups.containsKey(group), "Group already started: %s", group);
+
+ SchedulingGroup info = new SchedulingGroup<>();
+ groups.put(group, info);
+ }
+
+ public Set finishGroup(G group)
+ {
+ SchedulingGroup info = groups.remove(group);
+ checkArgument(info != null, "Unknown group: %s", group);
+
+ runnableQueue.removeIfPresent(group);
+ baselineWeights.removeIfPresent(group);
+ return info.tasks();
+ }
+
+ public boolean containsGroup(G group)
+ {
+ return groups.containsKey(group);
+ }
+
+ public Set finishAll()
+ {
+ Set groups = ImmutableSet.copyOf(this.groups.keySet());
+ return groups.stream()
+ .map(this::finishGroup)
+ .flatMap(Collection::stream)
+ .collect(Collectors.toSet());
+ }
+
+ public void finish(G group, T task)
+ {
+ checkArgument(groups.containsKey(group), "Unknown group: %s", group);
+
+ SchedulingGroup info = groups.get(group);
+
+ State previousState = info.state();
+ info.finish(task);
+ State newState = info.state();
+
+ if (newState == RUNNABLE) {
+ runnableQueue.addOrReplace(group, info.weight());
+ baselineWeights.addOrReplace(group, info.weight());
+ }
+ else if (newState == RUNNING) {
+ runnableQueue.removeIfPresent(group);
+ baselineWeights.addOrReplace(group, info.weight());
+ }
+ else if (newState == BLOCKED && previousState != BLOCKED) {
+ info.addWeight(-baselineWeight());
+ runnableQueue.removeIfPresent(group);
+ baselineWeights.removeIfPresent(group);
+ }
+
+ verifyState(group);
+ }
+
+ public void enqueue(G group, T task, long deltaWeight)
+ {
+ checkArgument(groups.containsKey(group), "Unknown group: %s", group);
+
+ SchedulingGroup info = groups.get(group);
+
+ State previousState = info.state();
+ info.enqueue(task, deltaWeight);
+ verify(info.state() == RUNNABLE);
+
+ if (previousState == BLOCKED) {
+ // When transitioning from blocked, set the baseline weight to the minimum current weight
+ // to avoid the newly unblocked group from monopolizing the queue while it catches up
+ info.addWeight(baselineWeight());
+ }
+
+ runnableQueue.addOrReplace(group, info.weight());
+ baselineWeights.addOrReplace(group, info.weight());
+
+ verifyState(group);
+ }
+
+ public void block(G group, T task, long deltaWeight)
+ {
+ SchedulingGroup info = groups.get(group);
+ checkArgument(info != null, "Unknown group: %s", group);
+ checkArgument(info.state() == RUNNABLE || info.state() == RUNNING, "Group is already blocked: %s", group);
+
+ State previousState = info.state();
+ info.block(task, deltaWeight);
+
+ doTransition(group, info, previousState, info.state());
+ }
+
+ public T dequeue(long expectedWeight)
+ {
+ G group = runnableQueue.poll();
+
+ if (group == null) {
+ return null;
+ }
+
+ SchedulingGroup info = groups.get(group);
+ verify(info.state() == RUNNABLE, "Group is not runnable: %s", group);
+
+ T task = info.dequeue(expectedWeight);
+ verify(task != null);
+
+ baselineWeights.addOrReplace(group, info.weight());
+ if (info.state() == RUNNABLE) {
+ runnableQueue.add(group, info.weight());
+ }
+
+ checkState(info.state() == RUNNABLE || info.state() == RUNNING);
+ verifyState(group);
+
+ return task;
+ }
+
+ public T peek()
+ {
+ G group = runnableQueue.peek();
+
+ if (group == null) {
+ return null;
+ }
+
+ SchedulingGroup info = groups.get(group);
+ verify(info.state() == RUNNABLE, "Group is not runnable: %s", group);
+
+ T task = info.peek();
+ checkState(task != null);
+
+ return task;
+ }
+
+ public int getRunnableCount()
+ {
+ return runnableQueue.values().stream()
+ .map(groups::get)
+ .mapToInt(SchedulingGroup::runnableCount)
+ .sum();
+ }
+
+ public State state(G group)
+ {
+ SchedulingGroup info = groups.get(group);
+ checkArgument(info != null, "Unknown group: %s", group);
+
+ return info.state();
+ }
+
+ private long baselineWeight()
+ {
+ if (baselineWeights.isEmpty()) {
+ return 0;
+ }
+
+ return baselineWeights.nextPriority();
+ }
+
+ private void doTransition(G group, SchedulingGroup info, State previousState, State newState)
+ {
+ if (newState == RUNNABLE) {
+ runnableQueue.addOrReplace(group, info.weight());
+ baselineWeights.addOrReplace(group, info.weight());
+ }
+ else if (newState == RUNNING) {
+ runnableQueue.removeIfPresent(group);
+ baselineWeights.addOrReplace(group, info.weight());
+ }
+ else if (newState == BLOCKED && previousState != BLOCKED) {
+ info.addWeight(-baselineWeight());
+ runnableQueue.removeIfPresent(group);
+ baselineWeights.removeIfPresent(group);
+ }
+
+ verifyState(group);
+ }
+
+ private void verifyState(G groupKey)
+ {
+ SchedulingGroup group = groups.get(groupKey);
+ checkArgument(group != null, "Unknown group: %s", groupKey);
+
+ switch (group.state()) {
+ case BLOCKED -> {
+ checkState(!runnableQueue.contains(groupKey), "Group in BLOCKED state should not be in queue: %s", groupKey);
+ checkState(!baselineWeights.contains(groupKey));
+ }
+ case RUNNABLE -> {
+ checkState(runnableQueue.contains(groupKey), "Group in RUNNABLE state should be in queue: %s", groupKey);
+ checkState(baselineWeights.contains(groupKey));
+ }
+ case RUNNING -> {
+ checkState(!runnableQueue.contains(groupKey), "Group in RUNNING state should not be in queue: %s", groupKey);
+ checkState(baselineWeights.contains(groupKey));
+ }
+ }
+ }
+
+ @Override
+ public String toString()
+ {
+ StringBuilder builder = new StringBuilder();
+
+ builder.append("Baseline weight: %s\n".formatted(baselineWeight()));
+ builder.append("\n");
+
+ for (Map.Entry> entry : groups.entrySet()) {
+ G group = entry.getKey();
+ SchedulingGroup info = entry.getValue();
+
+ String prefix = "%s %s".formatted(
+ group == runnableQueue.peek() ? "=>" : " -",
+ group);
+
+ String details = switch (entry.getValue().state()) {
+ case BLOCKED -> "[BLOCKED, saved delta = %s]".formatted(info.weight());
+ case RUNNING, RUNNABLE -> "[%s, weight = %s, baseline = %s]".formatted(info.state(), info.weight(), info.baselineWeight());
+ };
+
+ builder.append((prefix + " " + details).indent(4));
+ builder.append(info.toString().indent(8));
+ }
+
+ return builder.toString();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/State.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/State.java
new file mode 100644
index 000000000000..35e9bca8a6d6
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/State.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+enum State
+{
+ BLOCKED, // all tasks are blocked
+ RUNNING, // all tasks are dequeued and running
+ RUNNABLE // some tasks are enqueued and ready to run
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Task.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Task.java
new file mode 100644
index 000000000000..af62f7e4fb1d
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/Task.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import io.trino.annotation.NotThreadSafe;
+
+@NotThreadSafe
+final class Task
+{
+ private State state;
+ private long weight;
+ private long uncommittedWeight;
+
+ public Task(long initialWeight)
+ {
+ weight = initialWeight;
+ }
+
+ public void setState(State state)
+ {
+ this.state = state;
+ }
+
+ public void commitWeight(long delta)
+ {
+ weight += delta;
+ uncommittedWeight = 0;
+ }
+
+ public void addWeight(long delta)
+ {
+ weight += delta;
+ }
+
+ public long weight()
+ {
+ return weight + uncommittedWeight;
+ }
+
+ public void setUncommittedWeight(long weight)
+ {
+ this.uncommittedWeight = weight;
+ }
+
+ public long uncommittedWeight()
+ {
+ return uncommittedWeight;
+ }
+
+ public State state()
+ {
+ return state;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/TaskControl.java b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/TaskControl.java
new file mode 100644
index 000000000000..9375ca0b6b0e
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/TaskControl.java
@@ -0,0 +1,366 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.base.Ticker;
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Equality is based on group and id for the purpose of adding to the scheduling queue.
+ */
+@ThreadSafe
+final class TaskControl
+{
+ private final Group group;
+ private final int id;
+ private final Ticker ticker;
+
+ private final Lock lock = new ReentrantLock();
+
+ @GuardedBy("lock")
+ private final Condition wakeup = lock.newCondition();
+
+ @GuardedBy("lock")
+ private boolean ready;
+
+ @GuardedBy("lock")
+ private boolean blocked;
+
+ @GuardedBy("lock")
+ private boolean cancelled;
+
+ @GuardedBy("lock")
+ private State state;
+
+ private volatile long periodStart;
+ private final AtomicLong startNanos = new AtomicLong();
+ private final AtomicLong scheduledNanos = new AtomicLong();
+ private final AtomicLong blockedNanos = new AtomicLong();
+ private final AtomicLong waitNanos = new AtomicLong();
+ private volatile Thread thread;
+
+ public TaskControl(Group group, int id, Ticker ticker)
+ {
+ this.group = requireNonNull(group, "group is null");
+ this.id = id;
+ this.ticker = requireNonNull(ticker, "ticker is null");
+ this.state = State.NEW;
+ this.ready = false;
+ this.periodStart = ticker.read();
+ }
+
+ public void setThread(Thread thread)
+ {
+ this.thread = thread;
+ }
+
+ public void cancel()
+ {
+ lock.lock();
+ try {
+ cancelled = true;
+ wakeup.signal();
+
+ // TODO: it should be possible to interrupt the thread, but
+ // it appears that it's not safe to do so. It can cause the query
+ // to get stuck (e.g., AbstractDistributedEngineOnlyQueries.testSelectiveLimit)
+ //
+ // Thread thread = this.thread;
+ // if (thread != null) {
+ // thread.interrupt();
+ // }
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * Called by the scheduler thread when the task is ready to run. It
+ * causes anyone blocking in {@link #awaitReady()} to wake up.
+ *
+ * @return false if the task was already cancelled
+ */
+ public boolean markReady()
+ {
+ lock.lock();
+ try {
+ if (cancelled) {
+ return false;
+ }
+ ready = true;
+ wakeup.signal();
+ }
+ finally {
+ lock.unlock();
+ }
+
+ return true;
+ }
+
+ public void markNotReady()
+ {
+ lock.lock();
+ try {
+ ready = false;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public boolean isReady()
+ {
+ lock.lock();
+ try {
+ return ready;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * @return false if the operation was interrupted due to cancellation
+ */
+ public boolean awaitReady()
+ {
+ lock.lock();
+ try {
+ while (!ready && !cancelled) {
+ try {
+ wakeup.await();
+ }
+ catch (InterruptedException e) {
+ }
+ }
+
+ return !cancelled;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public void markUnblocked()
+ {
+ lock.lock();
+ try {
+ blocked = false;
+ wakeup.signal();
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public void markBlocked()
+ {
+ lock.lock();
+ try {
+ blocked = true;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public void awaitUnblock()
+ {
+ lock.lock();
+ try {
+ while (blocked && !cancelled) {
+ try {
+ wakeup.await();
+ }
+ catch (InterruptedException e) {
+ }
+ }
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ /**
+ * @return false if the transition was unsuccessful due to the task being interrupted
+ */
+ public boolean transitionToBlocked()
+ {
+ boolean success = transitionTo(State.BLOCKED);
+
+ if (success) {
+ markBlocked();
+ }
+
+ return success;
+ }
+
+ public void transitionToFinished()
+ {
+ transitionTo(State.FINISHED);
+ }
+
+ /**
+ * @return false if the transition was unsuccessful due to the task being interrupted
+ */
+ public boolean transitionToWaiting()
+ {
+ boolean success = transitionTo(State.WAITING);
+
+ if (success) {
+ markNotReady();
+ }
+
+ return success;
+ }
+
+ /**
+ * @return false if the transition was unsuccessful due to the task being interrupted
+ */
+ public boolean transitionToRunning()
+ {
+ return transitionTo(State.RUNNING);
+ }
+
+ private boolean transitionTo(State state)
+ {
+ lock.lock();
+ try {
+ recordPeriodEnd(this.state);
+
+ if (cancelled) {
+ this.state = State.INTERRUPTED;
+ return false;
+ }
+ else {
+ this.state = state;
+ return true;
+ }
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ private void recordPeriodEnd(State state)
+ {
+ long now = ticker.read();
+ long elapsed = now - periodStart;
+ switch (state) {
+ case RUNNING -> scheduledNanos.addAndGet(elapsed);
+ case BLOCKED -> blockedNanos.addAndGet(elapsed);
+ case NEW -> startNanos.addAndGet(elapsed);
+ case WAITING -> waitNanos.addAndGet(elapsed);
+ case INTERRUPTED, FINISHED -> {}
+ }
+ periodStart = now;
+ }
+
+ public Group group()
+ {
+ return group;
+ }
+
+ public State getState()
+ {
+ lock.lock();
+ try {
+ return state;
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public long elapsed()
+ {
+ return ticker.read() - periodStart;
+ }
+
+ public long getStartNanos()
+ {
+ return startNanos.get();
+ }
+
+ public long getWaitNanos()
+ {
+ return waitNanos.get();
+ }
+
+ public long getScheduledNanos()
+ {
+ return scheduledNanos.get();
+ }
+
+ public long getBlockedNanos()
+ {
+ return blockedNanos.get();
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ TaskControl that = (TaskControl) o;
+ return id == that.id && group.equals(that.group);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(group, id);
+ }
+
+ @Override
+ public String toString()
+ {
+ lock.lock();
+ try {
+ return group.name() + "-" + id + " [" + state + "]";
+ }
+ finally {
+ lock.unlock();
+ }
+ }
+
+ public Thread getThread()
+ {
+ return thread;
+ }
+
+ public enum State
+ {
+ NEW,
+ WAITING,
+ RUNNING,
+ BLOCKED,
+ INTERRUPTED,
+ FINISHED
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/group-state-diagram.dot b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/group-state-diagram.dot
new file mode 100644
index 000000000000..bc1346753ee0
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/scheduler/group-state-diagram.dot
@@ -0,0 +1,24 @@
+digraph Group {
+ node [shape=box];
+
+ start [shape=point];
+ split1 [shape=point];
+ split2 [shape=point];
+
+
+ start -> blocked;
+ blocked -> runnable [label="enqueueTask()"];
+ runnable -> runnable [label="enqueueTask()\nblockTask()"];
+ runnable -> split1 [label="dequeueTask()"];
+ split1 -> runnable [label="queue not empty"];
+ split1 -> running [label="queue empty"];
+ running -> split2 [label="blockTask()"];
+ running -> runnable [label="enqueueTask()"];
+ split2 -> blocked [label="all blocked"];
+ split2 -> running [label="not all blocked"];
+ blocked -> blocked [label="finishTask()"];
+ running -> running [label="finishTask()"];
+ runnable -> split3 [label="finishTask()"];
+ split3 -> blocked [label="all blocked"];
+ split3 -> running [label="all running"];
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/MultilevelSplitQueue.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/MultilevelSplitQueue.java
similarity index 99%
rename from core/trino-main/src/main/java/io/trino/execution/executor/MultilevelSplitQueue.java
rename to core/trino-main/src/main/java/io/trino/execution/executor/timesharing/MultilevelSplitQueue.java
index 317e28bdbd09..70403cb6c7ca 100644
--- a/core/trino-main/src/main/java/io/trino/execution/executor/MultilevelSplitQueue.java
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/MultilevelSplitQueue.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.annotations.VisibleForTesting;
import com.google.errorprone.annotations.ThreadSafe;
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/PrioritizedSplitRunner.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/PrioritizedSplitRunner.java
similarity index 97%
rename from core/trino-main/src/main/java/io/trino/execution/executor/PrioritizedSplitRunner.java
rename to core/trino-main/src/main/java/io/trino/execution/executor/timesharing/PrioritizedSplitRunner.java
index 1c3b5d15de9e..157c6cdd0cd9 100644
--- a/core/trino-main/src/main/java/io/trino/execution/executor/PrioritizedSplitRunner.java
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/PrioritizedSplitRunner.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.base.Ticker;
import com.google.common.util.concurrent.ListenableFuture;
@@ -50,7 +50,7 @@ public final class PrioritizedSplitRunner
private final long createdNanos = System.nanoTime();
- private final TaskHandle taskHandle;
+ private final TimeSharingTaskHandle taskHandle;
private final int splitId;
private final long workerId;
private final SplitRunner split;
@@ -81,7 +81,7 @@ public final class PrioritizedSplitRunner
private final TimeStat unblockedQuantaWallTime;
PrioritizedSplitRunner(
- TaskHandle taskHandle,
+ TimeSharingTaskHandle taskHandle,
int splitId,
SplitRunner split,
Span splitSpan,
@@ -107,7 +107,7 @@ public final class PrioritizedSplitRunner
updateLevelPriority();
}
- public TaskHandle getTaskHandle()
+ public TimeSharingTaskHandle getTaskHandle()
{
return taskHandle;
}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/Priority.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/Priority.java
similarity index 97%
rename from core/trino-main/src/main/java/io/trino/execution/executor/Priority.java
rename to core/trino-main/src/main/java/io/trino/execution/executor/timesharing/Priority.java
index 180f698742ae..a7c0ac0c9f82 100644
--- a/core/trino-main/src/main/java/io/trino/execution/executor/Priority.java
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/Priority.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.errorprone.annotations.Immutable;
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java
new file mode 100644
index 000000000000..d9b623481bfb
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutor.java
@@ -0,0 +1,948 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.timesharing;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import com.google.inject.Inject;
+import io.airlift.concurrent.SetThreadName;
+import io.airlift.concurrent.ThreadPoolExecutorMBean;
+import io.airlift.log.Logger;
+import io.airlift.stats.CounterStat;
+import io.airlift.stats.DistributionStat;
+import io.airlift.stats.TimeDistribution;
+import io.airlift.stats.TimeStat;
+import io.airlift.units.Duration;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.context.Context;
+import io.trino.execution.SplitRunner;
+import io.trino.execution.TaskId;
+import io.trino.execution.TaskManagerConfig;
+import io.trino.execution.executor.RunningSplitInfo;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.TaskHandle;
+import io.trino.spi.TrinoException;
+import io.trino.spi.VersionEmbedder;
+import io.trino.tracing.TrinoAttributes;
+import jakarta.annotation.PostConstruct;
+import jakarta.annotation.PreDestroy;
+import org.weakref.jmx.Managed;
+import org.weakref.jmx.Nested;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.OptionalInt;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongArray;
+import java.util.function.DoubleSupplier;
+import java.util.function.Predicate;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
+import static com.google.common.collect.ImmutableSet.toImmutableSet;
+import static com.google.common.collect.Sets.newConcurrentHashSet;
+import static io.airlift.concurrent.Threads.threadsNamed;
+import static io.airlift.tracing.Tracing.noopTracer;
+import static io.trino.execution.executor.timesharing.MultilevelSplitQueue.computeLevel;
+import static io.trino.version.EmbedVersion.testingVersionEmbedder;
+import static java.lang.Math.min;
+import static java.lang.String.format;
+import static java.util.Objects.requireNonNull;
+import static java.util.concurrent.Executors.newCachedThreadPool;
+import static java.util.concurrent.TimeUnit.MICROSECONDS;
+import static java.util.concurrent.TimeUnit.NANOSECONDS;
+
+@ThreadSafe
+public class TimeSharingTaskExecutor
+ implements TaskExecutor
+{
+ private static final Logger log = Logger.get(TimeSharingTaskExecutor.class);
+ private static final AtomicLong NEXT_RUNNER_ID = new AtomicLong();
+
+ private final ExecutorService executor;
+ private final ThreadPoolExecutorMBean executorMBean;
+
+ private final int runnerThreads;
+ private final int minimumNumberOfDrivers;
+ private final int guaranteedNumberOfDriversPerTask;
+ private final int maximumNumberOfDriversPerTask;
+ private final VersionEmbedder versionEmbedder;
+ private final Tracer tracer;
+
+ private final Ticker ticker;
+
+ private final Duration stuckSplitsWarningThreshold;
+ private final SortedSet runningSplitInfos = new ConcurrentSkipListSet<>();
+
+ @GuardedBy("this")
+ private final List tasks;
+
+ /**
+ * All splits registered with the task executor.
+ */
+ @GuardedBy("this")
+ private final Set allSplits = new HashSet<>();
+
+ /**
+ * Intermediate splits (i.e. splits that should not be queued).
+ */
+ @GuardedBy("this")
+ private final Set intermediateSplits = new HashSet<>();
+
+ /**
+ * Splits waiting for a runner thread.
+ */
+ private final MultilevelSplitQueue waitingSplits;
+
+ /**
+ * Splits running on a thread.
+ */
+ private final Set runningSplits = newConcurrentHashSet();
+
+ /**
+ * Splits blocked by the driver.
+ */
+ private final Map> blockedSplits = new ConcurrentHashMap<>();
+
+ private final AtomicLongArray completedTasksPerLevel = new AtomicLongArray(5);
+ private final AtomicLongArray completedSplitsPerLevel = new AtomicLongArray(5);
+
+ private final TimeStat splitQueuedTime = new TimeStat(NANOSECONDS);
+ private final TimeStat splitWallTime = new TimeStat(NANOSECONDS);
+
+ private final TimeDistribution leafSplitWallTime = new TimeDistribution(MICROSECONDS);
+ private final TimeDistribution intermediateSplitWallTime = new TimeDistribution(MICROSECONDS);
+
+ private final TimeDistribution leafSplitScheduledTime = new TimeDistribution(MICROSECONDS);
+ private final TimeDistribution intermediateSplitScheduledTime = new TimeDistribution(MICROSECONDS);
+
+ private final TimeDistribution leafSplitWaitTime = new TimeDistribution(MICROSECONDS);
+ private final TimeDistribution intermediateSplitWaitTime = new TimeDistribution(MICROSECONDS);
+
+ private final TimeDistribution leafSplitCpuTime = new TimeDistribution(MICROSECONDS);
+ private final TimeDistribution intermediateSplitCpuTime = new TimeDistribution(MICROSECONDS);
+
+ // shared between SplitRunners
+ private final CounterStat globalCpuTimeMicros = new CounterStat();
+ private final CounterStat globalScheduledTimeMicros = new CounterStat();
+
+ private final TimeStat blockedQuantaWallTime = new TimeStat(MICROSECONDS);
+ private final TimeStat unblockedQuantaWallTime = new TimeStat(MICROSECONDS);
+
+ private final DistributionStat leafSplitsSize = new DistributionStat();
+ @GuardedBy("this")
+ private long lastLeafSplitsSizeRecordTime;
+ @GuardedBy("this")
+ private long lastLeafSplitsSize;
+
+ private volatile boolean closed;
+
+ @Inject
+ public TimeSharingTaskExecutor(TaskManagerConfig config, VersionEmbedder versionEmbedder, Tracer tracer, MultilevelSplitQueue splitQueue)
+ {
+ this(
+ config.getMaxWorkerThreads(),
+ config.getMinDrivers(),
+ config.getMinDriversPerTask(),
+ config.getMaxDriversPerTask(),
+ config.getInterruptStuckSplitTasksWarningThreshold(),
+ versionEmbedder,
+ tracer,
+ splitQueue,
+ Ticker.systemTicker());
+ }
+
+ @VisibleForTesting
+ public TimeSharingTaskExecutor(int runnerThreads, int minDrivers, int guaranteedNumberOfDriversPerTask, int maximumNumberOfDriversPerTask, Ticker ticker)
+ {
+ this(runnerThreads, minDrivers, guaranteedNumberOfDriversPerTask, maximumNumberOfDriversPerTask, new Duration(10, TimeUnit.MINUTES), testingVersionEmbedder(), noopTracer(), new MultilevelSplitQueue(2), ticker);
+ }
+
+ @VisibleForTesting
+ public TimeSharingTaskExecutor(int runnerThreads, int minDrivers, int guaranteedNumberOfDriversPerTask, int maximumNumberOfDriversPerTask, MultilevelSplitQueue splitQueue, Ticker ticker)
+ {
+ this(runnerThreads, minDrivers, guaranteedNumberOfDriversPerTask, maximumNumberOfDriversPerTask, new Duration(10, TimeUnit.MINUTES), testingVersionEmbedder(), noopTracer(), splitQueue, ticker);
+ }
+
+ @VisibleForTesting
+ public TimeSharingTaskExecutor(
+ int runnerThreads,
+ int minDrivers,
+ int guaranteedNumberOfDriversPerTask,
+ int maximumNumberOfDriversPerTask,
+ Duration stuckSplitsWarningThreshold,
+ VersionEmbedder versionEmbedder,
+ Tracer tracer,
+ MultilevelSplitQueue splitQueue,
+ Ticker ticker)
+ {
+ checkArgument(runnerThreads > 0, "runnerThreads must be at least 1");
+ checkArgument(guaranteedNumberOfDriversPerTask > 0, "guaranteedNumberOfDriversPerTask must be at least 1");
+ checkArgument(maximumNumberOfDriversPerTask > 0, "maximumNumberOfDriversPerTask must be at least 1");
+ checkArgument(guaranteedNumberOfDriversPerTask <= maximumNumberOfDriversPerTask, "guaranteedNumberOfDriversPerTask cannot be greater than maximumNumberOfDriversPerTask");
+
+ // we manage thread pool size directly, so create an unlimited pool
+ this.executor = newCachedThreadPool(threadsNamed("task-processor-%s"));
+ this.executorMBean = new ThreadPoolExecutorMBean((ThreadPoolExecutor) executor);
+ this.runnerThreads = runnerThreads;
+ this.versionEmbedder = requireNonNull(versionEmbedder, "versionEmbedder is null");
+ this.tracer = requireNonNull(tracer, "tracer is null");
+
+ this.ticker = requireNonNull(ticker, "ticker is null");
+ this.stuckSplitsWarningThreshold = requireNonNull(stuckSplitsWarningThreshold, "stuckSplitsWarningThreshold is null");
+
+ this.minimumNumberOfDrivers = minDrivers;
+ this.guaranteedNumberOfDriversPerTask = guaranteedNumberOfDriversPerTask;
+ this.maximumNumberOfDriversPerTask = maximumNumberOfDriversPerTask;
+ this.waitingSplits = requireNonNull(splitQueue, "splitQueue is null");
+ this.tasks = new LinkedList<>();
+ this.lastLeafSplitsSizeRecordTime = ticker.read();
+ }
+
+ @PostConstruct
+ @Override
+ public synchronized void start()
+ {
+ checkState(!closed, "TaskExecutor is closed");
+ for (int i = 0; i < runnerThreads; i++) {
+ addRunnerThread();
+ }
+ }
+
+ @PreDestroy
+ @Override
+ public synchronized void stop()
+ {
+ closed = true;
+ executor.shutdownNow();
+ }
+
+ @Override
+ public synchronized String toString()
+ {
+ return toStringHelper(this)
+ .add("runnerThreads", runnerThreads)
+ .add("allSplits", allSplits.size())
+ .add("intermediateSplits", intermediateSplits.size())
+ .add("waitingSplits", waitingSplits.size())
+ .add("runningSplits", runningSplits.size())
+ .add("blockedSplits", blockedSplits.size())
+ .toString();
+ }
+
+ private synchronized void addRunnerThread()
+ {
+ try {
+ executor.execute(versionEmbedder.embedVersion(new TaskRunner()));
+ }
+ catch (RejectedExecutionException ignored) {
+ }
+ }
+
+ @Override
+ public synchronized TimeSharingTaskHandle addTask(
+ TaskId taskId,
+ DoubleSupplier utilizationSupplier,
+ int initialSplitConcurrency,
+ Duration splitConcurrencyAdjustFrequency,
+ OptionalInt maxDriversPerTask)
+ {
+ requireNonNull(taskId, "taskId is null");
+ requireNonNull(utilizationSupplier, "utilizationSupplier is null");
+ checkArgument(maxDriversPerTask.isEmpty() || maxDriversPerTask.getAsInt() <= maximumNumberOfDriversPerTask,
+ "maxDriversPerTask cannot be greater than the configured value");
+
+ log.debug("Task scheduled %s", taskId);
+
+ TimeSharingTaskHandle taskHandle = new TimeSharingTaskHandle(taskId, waitingSplits, utilizationSupplier, initialSplitConcurrency, splitConcurrencyAdjustFrequency, maxDriversPerTask);
+
+ tasks.add(taskHandle);
+ return taskHandle;
+ }
+
+ @Override
+ public void removeTask(TaskHandle taskHandle)
+ {
+ TimeSharingTaskHandle handle = (TimeSharingTaskHandle) taskHandle;
+ try (SetThreadName ignored = new SetThreadName("Task-%s", handle.getTaskId())) {
+ // Skip additional scheduling if the task was already destroyed
+ if (!doRemoveTask(handle)) {
+ return;
+ }
+ }
+
+ // replace blocked splits that were terminated
+ synchronized (this) {
+ addNewEntrants();
+ recordLeafSplitsSize();
+ }
+ }
+
+ /**
+ * Returns true if the task handle was destroyed and removed splits as a result that may need to be replaced. Otherwise,
+ * if the {@link TimeSharingTaskHandle} was already destroyed or no splits were removed then this method returns false and no additional
+ * splits need to be scheduled.
+ */
+ private boolean doRemoveTask(TimeSharingTaskHandle taskHandle)
+ {
+ List splits;
+ synchronized (this) {
+ tasks.remove(taskHandle);
+
+ // Task is already destroyed
+ if (taskHandle.isDestroyed()) {
+ return false;
+ }
+
+ splits = taskHandle.destroy();
+ // stop tracking splits (especially blocked splits which may never unblock)
+ allSplits.removeAll(splits);
+ intermediateSplits.removeAll(splits);
+ blockedSplits.keySet().removeAll(splits);
+ waitingSplits.removeAll(splits);
+ recordLeafSplitsSize();
+ }
+
+ // call destroy outside of synchronized block as it is expensive and doesn't need a lock on the task executor
+ for (PrioritizedSplitRunner split : splits) {
+ split.destroy();
+ }
+
+ // record completed stats
+ long threadUsageNanos = taskHandle.getScheduledNanos();
+ completedTasksPerLevel.incrementAndGet(computeLevel(threadUsageNanos));
+
+ log.debug("Task finished or failed %s", taskHandle.getTaskId());
+ return !splits.isEmpty();
+ }
+
+ @Override
+ public List> enqueueSplits(TaskHandle taskHandle, boolean intermediate, List extends SplitRunner> taskSplits)
+ {
+ TimeSharingTaskHandle handle = (TimeSharingTaskHandle) taskHandle;
+ List splitsToDestroy = new ArrayList<>();
+ List> finishedFutures = new ArrayList<>(taskSplits.size());
+ synchronized (this) {
+ for (SplitRunner taskSplit : taskSplits) {
+ TaskId taskId = handle.getTaskId();
+ int splitId = handle.getNextSplitId();
+
+ Span splitSpan = tracer.spanBuilder(intermediate ? "split (intermediate)" : "split (leaf)")
+ .setParent(Context.current().with(taskSplit.getPipelineSpan()))
+ .setAttribute(TrinoAttributes.QUERY_ID, taskId.getQueryId().toString())
+ .setAttribute(TrinoAttributes.STAGE_ID, taskId.getStageId().toString())
+ .setAttribute(TrinoAttributes.TASK_ID, taskId.toString())
+ .setAttribute(TrinoAttributes.PIPELINE_ID, taskId.getStageId() + "-" + taskSplit.getPipelineId())
+ .setAttribute(TrinoAttributes.SPLIT_ID, taskId + "-" + splitId)
+ .startSpan();
+
+ PrioritizedSplitRunner prioritizedSplitRunner = new PrioritizedSplitRunner(
+ handle,
+ splitId,
+ taskSplit,
+ splitSpan,
+ tracer,
+ ticker,
+ globalCpuTimeMicros,
+ globalScheduledTimeMicros,
+ blockedQuantaWallTime,
+ unblockedQuantaWallTime);
+
+ if (intermediate) {
+ // add the runner to the handle so it can be destroyed if the task is canceled
+ if (handle.recordIntermediateSplit(prioritizedSplitRunner)) {
+ // Note: we do not record queued time for intermediate splits
+ startIntermediateSplit(prioritizedSplitRunner);
+ }
+ else {
+ splitsToDestroy.add(prioritizedSplitRunner);
+ }
+ }
+ else {
+ // add this to the work queue for the task
+ if (handle.enqueueSplit(prioritizedSplitRunner)) {
+ // if task is under the limit for guaranteed splits, start one
+ scheduleTaskIfNecessary(handle);
+ // if globally we have more resources, start more
+ addNewEntrants();
+ }
+ else {
+ splitsToDestroy.add(prioritizedSplitRunner);
+ }
+ }
+
+ finishedFutures.add(prioritizedSplitRunner.getFinishedFuture());
+ }
+ recordLeafSplitsSize();
+ }
+ for (PrioritizedSplitRunner split : splitsToDestroy) {
+ split.destroy();
+ }
+ return finishedFutures;
+ }
+
+ private void splitFinished(PrioritizedSplitRunner split)
+ {
+ completedSplitsPerLevel.incrementAndGet(split.getPriority().getLevel());
+ synchronized (this) {
+ allSplits.remove(split);
+
+ long wallNanos = System.nanoTime() - split.getCreatedNanos();
+ splitWallTime.add(Duration.succinctNanos(wallNanos));
+
+ if (intermediateSplits.remove(split)) {
+ intermediateSplitWallTime.add(wallNanos);
+ intermediateSplitScheduledTime.add(split.getScheduledNanos());
+ intermediateSplitWaitTime.add(split.getWaitNanos());
+ intermediateSplitCpuTime.add(split.getCpuTimeNanos());
+ }
+ else {
+ leafSplitWallTime.add(wallNanos);
+ leafSplitScheduledTime.add(split.getScheduledNanos());
+ leafSplitWaitTime.add(split.getWaitNanos());
+ leafSplitCpuTime.add(split.getCpuTimeNanos());
+ }
+
+ TimeSharingTaskHandle taskHandle = split.getTaskHandle();
+ taskHandle.splitComplete(split);
+
+ scheduleTaskIfNecessary(taskHandle);
+
+ addNewEntrants();
+ recordLeafSplitsSize();
+ }
+ // call destroy outside of synchronized block as it is expensive and doesn't need a lock on the task executor
+ split.destroy();
+ }
+
+ private synchronized void scheduleTaskIfNecessary(TimeSharingTaskHandle taskHandle)
+ {
+ // if task has less than the minimum guaranteed splits running,
+ // immediately schedule new splits for this task. This assures
+ // that a task gets its fair amount of consideration (you have to
+ // have splits to be considered for running on a thread).
+ int splitsToSchedule = min(guaranteedNumberOfDriversPerTask, taskHandle.getMaxDriversPerTask().orElse(Integer.MAX_VALUE)) - taskHandle.getRunningLeafSplits();
+ for (int i = 0; i < splitsToSchedule; ++i) {
+ PrioritizedSplitRunner split = taskHandle.pollNextSplit();
+ if (split == null) {
+ // no more splits to schedule
+ return;
+ }
+
+ startSplit(split);
+ splitQueuedTime.add(Duration.nanosSince(split.getCreatedNanos()));
+ }
+ recordLeafSplitsSize();
+ }
+
+ private synchronized void addNewEntrants()
+ {
+ // Ignore intermediate splits when checking minimumNumberOfDrivers.
+ // Otherwise with (for example) minimumNumberOfDrivers = 100, 200 intermediate splits
+ // and 100 leaf splits, depending on order of appearing splits, number of
+ // simultaneously running splits may vary. If leaf splits start first, there will
+ // be 300 running splits. If intermediate splits start first, there will be only
+ // 200 running splits.
+ int running = allSplits.size() - intermediateSplits.size();
+ for (int i = 0; i < minimumNumberOfDrivers - running; i++) {
+ PrioritizedSplitRunner split = pollNextSplitWorker();
+ if (split == null) {
+ break;
+ }
+
+ splitQueuedTime.add(Duration.nanosSince(split.getCreatedNanos()));
+ startSplit(split);
+ }
+ }
+
+ private synchronized void startIntermediateSplit(PrioritizedSplitRunner split)
+ {
+ startSplit(split);
+ intermediateSplits.add(split);
+ }
+
+ private synchronized void startSplit(PrioritizedSplitRunner split)
+ {
+ allSplits.add(split);
+ waitingSplits.offer(split);
+ }
+
+ private synchronized PrioritizedSplitRunner pollNextSplitWorker()
+ {
+ // todo find a better algorithm for this
+ // find the first task that produces a split, then move that task to the
+ // end of the task list, so we get round robin
+ for (Iterator iterator = tasks.iterator(); iterator.hasNext(); ) {
+ TimeSharingTaskHandle task = iterator.next();
+ // skip tasks that are already running the configured max number of drivers
+ if (task.getRunningLeafSplits() >= task.getMaxDriversPerTask().orElse(maximumNumberOfDriversPerTask)) {
+ continue;
+ }
+ PrioritizedSplitRunner split = task.pollNextSplit();
+ if (split != null) {
+ // move task to end of list
+ iterator.remove();
+
+ // CAUTION: we are modifying the list in the loop which would normally
+ // cause a ConcurrentModificationException but we exit immediately
+ tasks.add(task);
+ return split;
+ }
+ }
+ return null;
+ }
+
+ private synchronized void recordLeafSplitsSize()
+ {
+ long now = ticker.read();
+ long timeDifference = now - this.lastLeafSplitsSizeRecordTime;
+ if (timeDifference > 0) {
+ this.leafSplitsSize.add(lastLeafSplitsSize, timeDifference);
+ this.lastLeafSplitsSizeRecordTime = now;
+ }
+ // always record new lastLeafSplitsSize as it might have changed
+ // even if timeDifference is 0
+ this.lastLeafSplitsSize = allSplits.size() - intermediateSplits.size();
+ }
+
+ private class TaskRunner
+ implements Runnable
+ {
+ private final long runnerId = NEXT_RUNNER_ID.getAndIncrement();
+
+ @Override
+ public void run()
+ {
+ try (SetThreadName runnerName = new SetThreadName("SplitRunner-%s", runnerId)) {
+ while (!closed && !Thread.currentThread().isInterrupted()) {
+ // select next worker
+ PrioritizedSplitRunner split;
+ try {
+ split = waitingSplits.take();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ return;
+ }
+
+ String threadId = split.getTaskHandle().getTaskId() + "-" + split.getSplitId();
+ try (SetThreadName splitName = new SetThreadName(threadId)) {
+ RunningSplitInfo splitInfo = new RunningSplitInfo(ticker.read(), threadId, Thread.currentThread(), split.getTaskHandle().getTaskId(), split::getInfo);
+ runningSplitInfos.add(splitInfo);
+ runningSplits.add(split);
+
+ ListenableFuture blocked;
+ try {
+ blocked = split.process();
+ }
+ finally {
+ runningSplitInfos.remove(splitInfo);
+ runningSplits.remove(split);
+ }
+
+ if (split.isFinished()) {
+ if (log.isDebugEnabled()) {
+ log.debug("%s is finished", split.getInfo());
+ }
+ splitFinished(split);
+ }
+ else {
+ if (blocked.isDone()) {
+ waitingSplits.offer(split);
+ }
+ else {
+ blockedSplits.put(split, blocked);
+ blocked.addListener(() -> {
+ blockedSplits.remove(split);
+ // reset the level priority to prevent previously-blocked splits from starving existing splits
+ split.resetLevelPriority();
+ waitingSplits.offer(split);
+ }, executor);
+ }
+ }
+ }
+ catch (Throwable t) {
+ // ignore random errors due to driver thread interruption
+ if (!split.isDestroyed()) {
+ if (t instanceof TrinoException trinoException) {
+ log.error(t, "Error processing %s: %s: %s", split.getInfo(), trinoException.getErrorCode().getName(), trinoException.getMessage());
+ }
+ else {
+ log.error(t, "Error processing %s", split.getInfo());
+ }
+ }
+ splitFinished(split);
+ }
+ finally {
+ // Clear the interrupted flag on the current thread, driver cancellation may have triggered an interrupt
+ if (Thread.interrupted()) {
+ if (closed) {
+ // reset interrupted flag if closed before interrupt
+ Thread.currentThread().interrupt();
+ }
+ }
+ }
+ }
+ }
+ finally {
+ // unless we have been closed, we need to replace this thread
+ if (!closed) {
+ addRunnerThread();
+ }
+ }
+ }
+ }
+
+ //
+ // STATS
+ //
+
+ @Managed
+ public synchronized int getTasks()
+ {
+ return tasks.size();
+ }
+
+ @Managed
+ public int getRunnerThreads()
+ {
+ return runnerThreads;
+ }
+
+ @Managed
+ public int getMinimumNumberOfDrivers()
+ {
+ return minimumNumberOfDrivers;
+ }
+
+ @Managed
+ public synchronized int getTotalSplits()
+ {
+ return allSplits.size();
+ }
+
+ @Managed
+ public synchronized int getIntermediateSplits()
+ {
+ return intermediateSplits.size();
+ }
+
+ @Managed
+ public int getWaitingSplits()
+ {
+ return waitingSplits.size();
+ }
+
+ @Managed
+ @Nested
+ public DistributionStat getLeafSplitsSize()
+ {
+ return leafSplitsSize;
+ }
+
+ @Managed
+ public int getRunningSplits()
+ {
+ return runningSplits.size();
+ }
+
+ @Managed
+ public int getBlockedSplits()
+ {
+ return blockedSplits.size();
+ }
+
+ @Managed
+ public long getCompletedTasksLevel0()
+ {
+ return completedTasksPerLevel.get(0);
+ }
+
+ @Managed
+ public long getCompletedTasksLevel1()
+ {
+ return completedTasksPerLevel.get(1);
+ }
+
+ @Managed
+ public long getCompletedTasksLevel2()
+ {
+ return completedTasksPerLevel.get(2);
+ }
+
+ @Managed
+ public long getCompletedTasksLevel3()
+ {
+ return completedTasksPerLevel.get(3);
+ }
+
+ @Managed
+ public long getCompletedTasksLevel4()
+ {
+ return completedTasksPerLevel.get(4);
+ }
+
+ @Managed
+ public long getCompletedSplitsLevel0()
+ {
+ return completedSplitsPerLevel.get(0);
+ }
+
+ @Managed
+ public long getCompletedSplitsLevel1()
+ {
+ return completedSplitsPerLevel.get(1);
+ }
+
+ @Managed
+ public long getCompletedSplitsLevel2()
+ {
+ return completedSplitsPerLevel.get(2);
+ }
+
+ @Managed
+ public long getCompletedSplitsLevel3()
+ {
+ return completedSplitsPerLevel.get(3);
+ }
+
+ @Managed
+ public long getCompletedSplitsLevel4()
+ {
+ return completedSplitsPerLevel.get(4);
+ }
+
+ @Managed
+ public long getRunningTasksLevel0()
+ {
+ return getRunningTasksForLevel(0);
+ }
+
+ @Managed
+ public long getRunningTasksLevel1()
+ {
+ return getRunningTasksForLevel(1);
+ }
+
+ @Managed
+ public long getRunningTasksLevel2()
+ {
+ return getRunningTasksForLevel(2);
+ }
+
+ @Managed
+ public long getRunningTasksLevel3()
+ {
+ return getRunningTasksForLevel(3);
+ }
+
+ @Managed
+ public long getRunningTasksLevel4()
+ {
+ return getRunningTasksForLevel(4);
+ }
+
+ @Managed
+ @Nested
+ public TimeStat getSplitQueuedTime()
+ {
+ return splitQueuedTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat getSplitWallTime()
+ {
+ return splitWallTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat getBlockedQuantaWallTime()
+ {
+ return blockedQuantaWallTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeStat getUnblockedQuantaWallTime()
+ {
+ return unblockedQuantaWallTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getLeafSplitScheduledTime()
+ {
+ return leafSplitScheduledTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getIntermediateSplitScheduledTime()
+ {
+ return intermediateSplitScheduledTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getLeafSplitWallTime()
+ {
+ return leafSplitWallTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getIntermediateSplitWallTime()
+ {
+ return intermediateSplitWallTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getLeafSplitWaitTime()
+ {
+ return leafSplitWaitTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getIntermediateSplitWaitTime()
+ {
+ return intermediateSplitWaitTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getLeafSplitCpuTime()
+ {
+ return leafSplitCpuTime;
+ }
+
+ @Managed
+ @Nested
+ public TimeDistribution getIntermediateSplitCpuTime()
+ {
+ return intermediateSplitCpuTime;
+ }
+
+ @Managed
+ @Nested
+ public CounterStat getGlobalScheduledTimeMicros()
+ {
+ return globalScheduledTimeMicros;
+ }
+
+ @Managed
+ @Nested
+ public CounterStat getGlobalCpuTimeMicros()
+ {
+ return globalCpuTimeMicros;
+ }
+
+ private synchronized int getRunningTasksForLevel(int level)
+ {
+ int count = 0;
+ for (TimeSharingTaskHandle task : tasks) {
+ if (task.getPriority().getLevel() == level) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ public String getMaxActiveSplitsInfo()
+ {
+ // Sample output:
+ //
+ // 2 splits have been continuously active for more than 600.00ms seconds
+ //
+ // "20180907_054754_00000_88xi4.1.0-2" tid=99
+ // at java.util.Formatter$FormatSpecifier.(Formatter.java:2708)
+ // at java.util.Formatter.parse(Formatter.java:2560)
+ // at java.util.Formatter.format(Formatter.java:2501)
+ // at ... (more lines of stacktrace)
+ //
+ // "20180907_054754_00000_88xi4.1.0-3" tid=106
+ // at java.util.Formatter$FormatSpecifier.(Formatter.java:2709)
+ // at java.util.Formatter.parse(Formatter.java:2560)
+ // at java.util.Formatter.format(Formatter.java:2501)
+ // at ... (more line of stacktrace)
+ StringBuilder stackTrace = new StringBuilder();
+ int maxActiveSplitCount = 0;
+ String message = "%s splits have been continuously active for more than %s seconds\n";
+ for (RunningSplitInfo splitInfo : runningSplitInfos) {
+ Duration duration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
+ if (duration.compareTo(stuckSplitsWarningThreshold) >= 0) {
+ maxActiveSplitCount++;
+ stackTrace.append("\n");
+ stackTrace.append(format("\"%s\" tid=%s", splitInfo.getThreadId(), splitInfo.getThread().getId())).append("\n");
+ for (StackTraceElement traceElement : splitInfo.getThread().getStackTrace()) {
+ stackTrace.append("\tat ").append(traceElement).append("\n");
+ }
+ }
+ }
+
+ return format(message, maxActiveSplitCount, stuckSplitsWarningThreshold).concat(stackTrace.toString());
+ }
+
+ @Managed
+ public long getRunAwaySplitCount()
+ {
+ int count = 0;
+ for (RunningSplitInfo splitInfo : runningSplitInfos) {
+ Duration duration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
+ if (duration.compareTo(stuckSplitsWarningThreshold) > 0) {
+ count++;
+ }
+ }
+ return count;
+ }
+
+ @Override
+ public Set getStuckSplitTaskIds(Duration processingDurationThreshold, Predicate filter)
+ {
+ return runningSplitInfos.stream()
+ .filter((RunningSplitInfo splitInfo) -> {
+ Duration splitProcessingDuration = Duration.succinctNanos(ticker.read() - splitInfo.getStartTime());
+ return splitProcessingDuration.compareTo(processingDurationThreshold) > 0;
+ })
+ .filter(filter).map(RunningSplitInfo::getTaskId).collect(toImmutableSet());
+ }
+
+ @Managed(description = "Task processor executor")
+ @Nested
+ public ThreadPoolExecutorMBean getProcessorExecutor()
+ {
+ return executorMBean;
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskHandle.java b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskHandle.java
new file mode 100644
index 000000000000..6c235ba90b48
--- /dev/null
+++ b/core/trino-main/src/main/java/io/trino/execution/executor/timesharing/TimeSharingTaskHandle.java
@@ -0,0 +1,203 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.timesharing;
+
+import com.google.common.collect.ImmutableList;
+import com.google.errorprone.annotations.ThreadSafe;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import io.airlift.units.Duration;
+import io.trino.execution.SplitConcurrencyController;
+import io.trino.execution.TaskId;
+import io.trino.execution.executor.TaskHandle;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.OptionalInt;
+import java.util.Queue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.DoubleSupplier;
+
+import static com.google.common.base.MoreObjects.toStringHelper;
+import static java.util.Objects.requireNonNull;
+
+@ThreadSafe
+public class TimeSharingTaskHandle
+ implements TaskHandle
+{
+ private volatile boolean destroyed;
+ private final TaskId taskId;
+ private final DoubleSupplier utilizationSupplier;
+
+ @GuardedBy("this")
+ protected final Queue queuedLeafSplits = new ArrayDeque<>(10);
+ @GuardedBy("this")
+ protected final List runningLeafSplits = new ArrayList<>(10);
+ @GuardedBy("this")
+ protected final List runningIntermediateSplits = new ArrayList<>(10);
+ @GuardedBy("this")
+ protected long scheduledNanos;
+ @GuardedBy("this")
+ protected final SplitConcurrencyController concurrencyController;
+
+ private final AtomicInteger nextSplitId = new AtomicInteger();
+
+ private final AtomicReference priority = new AtomicReference<>(new Priority(0, 0));
+ private final MultilevelSplitQueue splitQueue;
+ private final OptionalInt maxDriversPerTask;
+
+ public TimeSharingTaskHandle(
+ TaskId taskId,
+ MultilevelSplitQueue splitQueue,
+ DoubleSupplier utilizationSupplier,
+ int initialSplitConcurrency,
+ Duration splitConcurrencyAdjustFrequency,
+ OptionalInt maxDriversPerTask)
+ {
+ this.taskId = requireNonNull(taskId, "taskId is null");
+ this.splitQueue = requireNonNull(splitQueue, "splitQueue is null");
+ this.utilizationSupplier = requireNonNull(utilizationSupplier, "utilizationSupplier is null");
+ this.maxDriversPerTask = requireNonNull(maxDriversPerTask, "maxDriversPerTask is null");
+ this.concurrencyController = new SplitConcurrencyController(
+ initialSplitConcurrency,
+ requireNonNull(splitConcurrencyAdjustFrequency, "splitConcurrencyAdjustFrequency is null"));
+ }
+
+ public synchronized Priority addScheduledNanos(long durationNanos)
+ {
+ concurrencyController.update(durationNanos, utilizationSupplier.getAsDouble(), runningLeafSplits.size());
+ scheduledNanos += durationNanos;
+
+ Priority newPriority = splitQueue.updatePriority(priority.get(), durationNanos, scheduledNanos);
+
+ priority.set(newPriority);
+ return newPriority;
+ }
+
+ public synchronized Priority resetLevelPriority()
+ {
+ Priority currentPriority = priority.get();
+ long levelMinPriority = splitQueue.getLevelMinPriority(currentPriority.getLevel(), scheduledNanos);
+
+ if (currentPriority.getLevelPriority() < levelMinPriority) {
+ Priority newPriority = new Priority(currentPriority.getLevel(), levelMinPriority);
+ priority.set(newPriority);
+ return newPriority;
+ }
+
+ return currentPriority;
+ }
+
+ @Override
+ public boolean isDestroyed()
+ {
+ return destroyed;
+ }
+
+ public Priority getPriority()
+ {
+ return priority.get();
+ }
+
+ public TaskId getTaskId()
+ {
+ return taskId;
+ }
+
+ public OptionalInt getMaxDriversPerTask()
+ {
+ return maxDriversPerTask;
+ }
+
+ // Returns any remaining splits. The caller must destroy these.
+ public synchronized List destroy()
+ {
+ destroyed = true;
+
+ ImmutableList.Builder builder = ImmutableList.builderWithExpectedSize(runningIntermediateSplits.size() + runningLeafSplits.size() + queuedLeafSplits.size());
+ builder.addAll(runningIntermediateSplits);
+ builder.addAll(runningLeafSplits);
+ builder.addAll(queuedLeafSplits);
+ runningIntermediateSplits.clear();
+ runningLeafSplits.clear();
+ queuedLeafSplits.clear();
+ return builder.build();
+ }
+
+ public synchronized boolean enqueueSplit(PrioritizedSplitRunner split)
+ {
+ if (destroyed) {
+ return false;
+ }
+ queuedLeafSplits.add(split);
+ return true;
+ }
+
+ public synchronized boolean recordIntermediateSplit(PrioritizedSplitRunner split)
+ {
+ if (destroyed) {
+ return false;
+ }
+ runningIntermediateSplits.add(split);
+ return true;
+ }
+
+ synchronized int getRunningLeafSplits()
+ {
+ return runningLeafSplits.size();
+ }
+
+ public synchronized long getScheduledNanos()
+ {
+ return scheduledNanos;
+ }
+
+ public synchronized PrioritizedSplitRunner pollNextSplit()
+ {
+ if (destroyed) {
+ return null;
+ }
+
+ if (runningLeafSplits.size() >= concurrencyController.getTargetConcurrency()) {
+ return null;
+ }
+
+ PrioritizedSplitRunner split = queuedLeafSplits.poll();
+ if (split != null) {
+ runningLeafSplits.add(split);
+ }
+ return split;
+ }
+
+ public synchronized void splitComplete(PrioritizedSplitRunner split)
+ {
+ concurrencyController.splitFinished(split.getScheduledNanos(), utilizationSupplier.getAsDouble(), runningLeafSplits.size());
+ runningIntermediateSplits.remove(split);
+ runningLeafSplits.remove(split);
+ }
+
+ public int getNextSplitId()
+ {
+ return nextSplitId.getAndIncrement();
+ }
+
+ @Override
+ public String toString()
+ {
+ return toStringHelper(this)
+ .add("taskId", taskId)
+ .toString();
+ }
+}
diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java b/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
index b2a4a27e88b7..a37b29e417cd 100644
--- a/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
+++ b/core/trino-main/src/main/java/io/trino/operator/DriverFactory.java
@@ -130,6 +130,7 @@ public Driver createDriver(DriverContext driverContext)
}
}
}
+ driverContext.failed(failure);
throw failure;
}
}
diff --git a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
index 8f4502fbe51a..5ce7a985d0ac 100644
--- a/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
+++ b/core/trino-main/src/main/java/io/trino/server/ServerMainModule.java
@@ -49,8 +49,10 @@
import io.trino.execution.TableExecuteContextManager;
import io.trino.execution.TaskManagementExecutor;
import io.trino.execution.TaskManagerConfig;
-import io.trino.execution.executor.MultilevelSplitQueue;
import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor;
+import io.trino.execution.executor.timesharing.MultilevelSplitQueue;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
import io.trino.execution.scheduler.NodeScheduler;
import io.trino.execution.scheduler.NodeSchedulerConfig;
import io.trino.execution.scheduler.TopologyAwareNodeSelectorModule;
@@ -282,8 +284,6 @@ protected void setup(Binder binder)
binder.bind(FailureInjector.class).in(Scopes.SINGLETON);
jaxrsBinder(binder).bind(TaskResource.class);
newExporter(binder).export(TaskResource.class).withGeneratedName();
- jaxrsBinder(binder).bind(TaskExecutorResource.class);
- newExporter(binder).export(TaskExecutorResource.class).withGeneratedName();
binder.bind(TaskManagementExecutor.class).in(Scopes.SINGLETON);
binder.bind(SqlTaskManager.class).in(Scopes.SINGLETON);
binder.bind(TableExecuteContextManager.class).in(Scopes.SINGLETON);
@@ -305,7 +305,7 @@ protected void setup(Binder binder)
binder.bind(LocalMemoryManagerExporter.class).in(Scopes.SINGLETON);
newOptionalBinder(binder, VersionEmbedder.class).setDefault().to(EmbedVersion.class).in(Scopes.SINGLETON);
newExporter(binder).export(SqlTaskManager.class).withGeneratedName();
- binder.bind(TaskExecutor.class).in(Scopes.SINGLETON);
+
newExporter(binder).export(TaskExecutor.class).withGeneratedName();
binder.bind(MultilevelSplitQueue.class).in(Scopes.SINGLETON);
newExporter(binder).export(MultilevelSplitQueue.class).withGeneratedName();
@@ -316,6 +316,24 @@ protected void setup(Binder binder)
binder.bind(PageFunctionCompiler.class).in(Scopes.SINGLETON);
newExporter(binder).export(PageFunctionCompiler.class).withGeneratedName();
configBinder(binder).bindConfig(TaskManagerConfig.class);
+
+ // TODO: use conditional module
+ TaskManagerConfig taskManagerConfig = buildConfigObject(TaskManagerConfig.class);
+ if (taskManagerConfig.isThreadPerDriverSchedulerEnabled()) {
+ binder.bind(TaskExecutor.class)
+ .to(ThreadPerDriverTaskExecutor.class)
+ .in(Scopes.SINGLETON);
+ }
+ else {
+ jaxrsBinder(binder).bind(TaskExecutorResource.class);
+ newExporter(binder).export(TaskExecutorResource.class).withGeneratedName();
+
+ binder.bind(TaskExecutor.class)
+ .to(TimeSharingTaskExecutor.class)
+ .in(Scopes.SINGLETON);
+ binder.bind(TimeSharingTaskExecutor.class).in(Scopes.SINGLETON);
+ }
+
if (retryPolicy == TASK) {
configBinder(binder).bindConfigDefaults(TaskManagerConfig.class, TaskManagerConfig::applyFaultTolerantExecutionDefaults);
}
diff --git a/core/trino-main/src/main/java/io/trino/server/TaskExecutorResource.java b/core/trino-main/src/main/java/io/trino/server/TaskExecutorResource.java
index 1248cb1515cf..0f8d19d52fba 100644
--- a/core/trino-main/src/main/java/io/trino/server/TaskExecutorResource.java
+++ b/core/trino-main/src/main/java/io/trino/server/TaskExecutorResource.java
@@ -14,7 +14,7 @@
package io.trino.server;
import com.google.inject.Inject;
-import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
import io.trino.server.security.ResourceSecurity;
import jakarta.ws.rs.GET;
import jakarta.ws.rs.Path;
@@ -27,11 +27,11 @@
@Path("/v1/maxActiveSplits")
public class TaskExecutorResource
{
- private final TaskExecutor taskExecutor;
+ private final TimeSharingTaskExecutor taskExecutor;
@Inject
public TaskExecutorResource(
- TaskExecutor taskExecutor)
+ TimeSharingTaskExecutor taskExecutor)
{
this.taskExecutor = requireNonNull(taskExecutor, "taskExecutor is null");
}
diff --git a/core/trino-main/src/main/java/io/trino/sql/gen/ExpressionProfiler.java b/core/trino-main/src/main/java/io/trino/sql/gen/ExpressionProfiler.java
index 7839d49b2d30..49f635be2ff4 100644
--- a/core/trino-main/src/main/java/io/trino/sql/gen/ExpressionProfiler.java
+++ b/core/trino-main/src/main/java/io/trino/sql/gen/ExpressionProfiler.java
@@ -19,7 +19,7 @@
import static com.google.common.base.Ticker.systemTicker;
import static com.google.common.base.Verify.verify;
-import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
+import static io.trino.execution.executor.timesharing.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.NANOSECONDS;
diff --git a/core/trino-main/src/main/java/io/trino/tracing/TrinoAttributes.java b/core/trino-main/src/main/java/io/trino/tracing/TrinoAttributes.java
index 31368e066e70..d32429a28f16 100644
--- a/core/trino-main/src/main/java/io/trino/tracing/TrinoAttributes.java
+++ b/core/trino-main/src/main/java/io/trino/tracing/TrinoAttributes.java
@@ -55,6 +55,8 @@ private TrinoAttributes() {}
public static final AttributeKey SPLIT_SCHEDULED_TIME_NANOS = longKey("trino.split.scheduled_time_nanos");
public static final AttributeKey SPLIT_CPU_TIME_NANOS = longKey("trino.split.cpu_time_nanos");
public static final AttributeKey SPLIT_WAIT_TIME_NANOS = longKey("trino.split.wait_time_nanos");
+ public static final AttributeKey SPLIT_START_TIME_NANOS = longKey("trino.split.start_time_nanos");
+ public static final AttributeKey SPLIT_BLOCK_TIME_NANOS = longKey("trino.split.block_time_nanos");
public static final AttributeKey SPLIT_BLOCKED = booleanKey("trino.split.blocked");
public static final AttributeKey EVENT_STATE = stringKey("state");
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManager.java b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java
similarity index 73%
rename from core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManager.java
rename to core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java
index 6015e4eb3ed3..58e468ea2a1f 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManager.java
+++ b/core/trino-main/src/test/java/io/trino/execution/BaseTestSqlTaskManager.java
@@ -13,16 +13,11 @@
*/
package io.trino.execution;
-import com.google.common.base.Ticker;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.ListenableFuture;
-import com.google.common.util.concurrent.SettableFuture;
-import com.google.errorprone.annotations.concurrent.GuardedBy;
import io.airlift.node.NodeInfo;
import io.airlift.stats.TestingGcMonitor;
-import io.airlift.testing.TestingTicker;
import io.airlift.units.DataSize;
import io.airlift.units.DataSize.Unit;
import io.airlift.units.Duration;
@@ -38,7 +33,6 @@
import io.trino.execution.buffer.PipelinedOutputBuffers;
import io.trino.execution.buffer.PipelinedOutputBuffers.OutputBufferId;
import io.trino.execution.executor.TaskExecutor;
-import io.trino.execution.executor.TaskHandle;
import io.trino.memory.LocalMemoryManager;
import io.trino.memory.NodeMemoryConfig;
import io.trino.memory.QueryContext;
@@ -53,21 +47,21 @@
import io.trino.spiller.LocalSpillManager;
import io.trino.spiller.NodeSpillConfig;
import io.trino.version.EmbedVersion;
-import org.testng.annotations.AfterClass;
-import org.testng.annotations.BeforeClass;
-import org.testng.annotations.Test;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInstance;
+import org.junit.jupiter.api.Timeout;
import java.net.URI;
import java.util.List;
import java.util.Optional;
-import java.util.OptionalInt;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import java.util.function.Predicate;
+import java.util.concurrent.atomic.AtomicInteger;
-import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static com.google.common.util.concurrent.MoreExecutors.directExecutor;
import static io.airlift.tracing.Tracing.noopTracer;
import static io.trino.SessionTestUtils.TEST_SESSION;
@@ -81,6 +75,7 @@
import static io.trino.execution.buffer.PipelinedOutputBuffers.BufferType.PARTITIONED;
import static io.trino.testing.TestingSession.testSessionBuilder;
import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotEquals;
@@ -88,27 +83,26 @@
import static org.testng.Assert.assertNull;
import static org.testng.Assert.assertTrue;
-public class TestSqlTaskManager
+@TestInstance(PER_CLASS)
+public abstract class BaseTestSqlTaskManager
{
- private static final TaskId TASK_ID = new TaskId(new StageId("query", 0), 1, 0);
public static final OutputBufferId OUT = new OutputBufferId(0);
+ private final AtomicInteger sequence = new AtomicInteger();
private TaskExecutor taskExecutor;
private TaskManagementExecutor taskManagementExecutor;
- private LocalMemoryManager localMemoryManager;
- private LocalSpillManager localSpillManager;
- @BeforeClass
+ protected abstract TaskExecutor createTaskExecutor();
+
+ @BeforeAll
public void setUp()
{
- localMemoryManager = new LocalMemoryManager(new NodeMemoryConfig());
- localSpillManager = new LocalSpillManager(new NodeSpillConfig());
- taskExecutor = new TaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
+ taskExecutor = createTaskExecutor();
taskExecutor.start();
taskManagementExecutor = new TaskManagementExecutor();
}
- @AfterClass(alwaysRun = true)
+ @AfterAll
public void tearDown()
{
taskExecutor.stop();
@@ -121,7 +115,7 @@ public void tearDown()
public void testEmptyQuery()
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
TaskInfo taskInfo = createTask(sqlTaskManager, taskId, PipelinedOutputBuffers.createInitial(PARTITIONED).withNoMoreBufferIds());
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
@@ -136,12 +130,13 @@ public void testEmptyQuery()
}
}
- @Test(timeOut = 30_000)
+ @Test
+ @Timeout(30)
public void testSimpleQuery()
throws Exception
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
createTask(sqlTaskManager, taskId, ImmutableSet.of(SPLIT), PipelinedOutputBuffers.createInitial(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds());
TaskInfo taskInfo = sqlTaskManager.getTaskInfo(taskId, TaskStatus.STARTING_VERSION).get();
@@ -174,7 +169,7 @@ public void testCancel()
throws InterruptedException, ExecutionException, TimeoutException
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
TaskInfo taskInfo = createTask(sqlTaskManager, taskId, PipelinedOutputBuffers.createInitial(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds());
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
assertNull(taskInfo.getStats().getEndTime());
@@ -198,7 +193,7 @@ public void testAbort()
throws InterruptedException, ExecutionException, TimeoutException
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
TaskInfo taskInfo = createTask(sqlTaskManager, taskId, PipelinedOutputBuffers.createInitial(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds());
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
assertNull(taskInfo.getStats().getEndTime());
@@ -217,12 +212,13 @@ public void testAbort()
}
}
- @Test(timeOut = 30_000)
+ @Test
+ @Timeout(30)
public void testAbortResults()
throws Exception
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig())) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
createTask(sqlTaskManager, taskId, ImmutableSet.of(SPLIT), PipelinedOutputBuffers.createInitial(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds());
TaskInfo taskInfo = sqlTaskManager.getTaskInfo(taskId, TaskStatus.STARTING_VERSION).get();
@@ -243,7 +239,7 @@ public void testRemoveOldTasks()
throws InterruptedException, ExecutionException, TimeoutException
{
try (SqlTaskManager sqlTaskManager = createSqlTaskManager(new TaskManagerConfig().setInfoMaxAge(new Duration(5, TimeUnit.MILLISECONDS)))) {
- TaskId taskId = TASK_ID;
+ TaskId taskId = newTaskId();
TaskInfo taskInfo = createTask(sqlTaskManager, taskId, PipelinedOutputBuffers.createInitial(PARTITIONED).withBuffer(OUT, 0).withNoMoreBufferIds());
assertEquals(taskInfo.getTaskStatus().getState(), TaskState.RUNNING);
@@ -263,58 +259,6 @@ public void testRemoveOldTasks()
}
}
- @Test
- public void testFailStuckSplitTasks()
- throws InterruptedException, ExecutionException, TimeoutException
- {
- TestingTicker ticker = new TestingTicker();
-
- TaskHandle taskHandle = taskExecutor.addTask(
- TASK_ID,
- () -> 1.0,
- 1,
- new Duration(1, SECONDS),
- OptionalInt.of(1));
- MockSplitRunner mockSplitRunner = new MockSplitRunner();
-
- TaskExecutor taskExecutor = new TaskExecutor(4, 8, 3, 4, ticker);
- // Here we explicitly enqueue an indefinite running split runner
- taskExecutor.enqueueSplits(taskHandle, false, ImmutableList.of(mockSplitRunner));
-
- taskExecutor.start();
- try {
- // wait for the task executor to start processing the split
- mockSplitRunner.waitForStart();
-
- TaskManagerConfig taskManagerConfig = new TaskManagerConfig()
- .setInterruptStuckSplitTasksEnabled(true)
- .setInterruptStuckSplitTasksDetectionInterval(new Duration(10, SECONDS))
- .setInterruptStuckSplitTasksWarningThreshold(new Duration(10, SECONDS))
- .setInterruptStuckSplitTasksTimeout(new Duration(10, SECONDS));
-
- try (SqlTaskManager sqlTaskManager = createSqlTaskManager(taskManagerConfig, new NodeMemoryConfig(), taskExecutor, stackTraceElements -> true)) {
- sqlTaskManager.addStateChangeListener(TASK_ID, (state) -> {
- if (state.isTerminatingOrDone() && !taskHandle.isDestroyed()) {
- taskExecutor.removeTask(taskHandle);
- }
- });
-
- ticker.increment(30, SECONDS);
- sqlTaskManager.failStuckSplitTasks();
-
- mockSplitRunner.waitForFinish();
- List taskInfos = sqlTaskManager.getAllTaskInfo();
- assertEquals(taskInfos.size(), 1);
-
- TaskInfo taskInfo = pollTerminatingTaskInfoUntilDone(sqlTaskManager, taskInfos.get(0));
- assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FAILED);
- }
- }
- finally {
- taskExecutor.stop();
- }
- }
-
@Test
public void testSessionPropertyMemoryLimitOverride()
{
@@ -382,43 +326,17 @@ private SqlTaskManager createSqlTaskManager(TaskManagerConfig taskManagerConfig,
taskExecutor,
createTestSplitMonitor(),
new NodeInfo("test"),
- localMemoryManager,
+ new LocalMemoryManager(nodeMemoryConfig),
taskManagementExecutor,
taskManagerConfig,
nodeMemoryConfig,
- localSpillManager,
+ new LocalSpillManager(new NodeSpillConfig()),
new NodeSpillConfig(),
new TestingGcMonitor(),
noopTracer(),
new ExchangeManagerRegistry());
}
- private SqlTaskManager createSqlTaskManager(
- TaskManagerConfig taskManagerConfig,
- NodeMemoryConfig nodeMemoryConfig,
- TaskExecutor taskExecutor,
- Predicate> stuckSplitStackTracePredicate)
- {
- return new SqlTaskManager(
- new EmbedVersion("testversion"),
- new NoConnectorServicesProvider(),
- createTestingPlanner(),
- new MockLocationFactory(),
- taskExecutor,
- createTestSplitMonitor(),
- new NodeInfo("test"),
- localMemoryManager,
- taskManagementExecutor,
- taskManagerConfig,
- nodeMemoryConfig,
- localSpillManager,
- new NodeSpillConfig(),
- new TestingGcMonitor(),
- noopTracer(),
- new ExchangeManagerRegistry(),
- stuckSplitStackTracePredicate);
- }
-
private TaskInfo createTask(SqlTaskManager sqlTaskManager, TaskId taskId, ImmutableSet splits, OutputBuffers outputBuffers)
{
return sqlTaskManager.updateTask(TEST_SESSION,
@@ -500,94 +418,6 @@ public URI createMemoryInfoLocation(InternalNode node)
}
}
- private static class MockSplitRunner
- implements SplitRunner
- {
- private final SettableFuture startedFuture = SettableFuture.create();
- private final SettableFuture finishedFuture = SettableFuture.create();
-
- @GuardedBy("this")
- private Thread runnerThread;
- @GuardedBy("this")
- private boolean closed;
-
- public void waitForStart()
- throws ExecutionException, InterruptedException, TimeoutException
- {
- startedFuture.get(10, SECONDS);
- }
-
- public void waitForFinish()
- throws ExecutionException, InterruptedException, TimeoutException
- {
- finishedFuture.get(10, SECONDS);
- }
-
- @Override
- public int getPipelineId()
- {
- return 0;
- }
-
- @Override
- public Span getPipelineSpan()
- {
- return Span.getInvalid();
- }
-
- @Override
- public synchronized boolean isFinished()
- {
- return closed;
- }
-
- @Override
- public ListenableFuture processFor(Duration duration)
- {
- startedFuture.set(null);
- synchronized (this) {
- runnerThread = Thread.currentThread();
-
- if (closed) {
- finishedFuture.set(null);
- return immediateVoidFuture();
- }
- }
-
- while (true) {
- try {
- Thread.sleep(100000);
- }
- catch (InterruptedException e) {
- break;
- }
- }
-
- synchronized (this) {
- closed = true;
- }
- finishedFuture.set(null);
-
- return immediateVoidFuture();
- }
-
- @Override
- public String getInfo()
- {
- return "MockSplitRunner";
- }
-
- @Override
- public synchronized void close()
- {
- closed = true;
-
- if (runnerThread != null) {
- runnerThread.interrupt();
- }
- }
- }
-
private static class NoConnectorServicesProvider
implements ConnectorServicesProvider
{
@@ -609,4 +439,9 @@ public ConnectorServices getConnectorServices(CatalogHandle catalogHandle)
throw new UnsupportedOperationException();
}
}
+
+ private TaskId newTaskId()
+ {
+ return new TaskId(new StageId("query" + sequence.incrementAndGet(), 0), 1, 0);
+ }
}
diff --git a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java
index c330906ccfcd..da666dac24e3 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java
+++ b/core/trino-main/src/test/java/io/trino/execution/TaskTestUtils.java
@@ -24,7 +24,7 @@
import io.trino.eventlistener.EventListenerConfig;
import io.trino.eventlistener.EventListenerManager;
import io.trino.exchange.ExchangeManagerRegistry;
-import io.trino.execution.TestSqlTaskManager.MockDirectExchangeClientSupplier;
+import io.trino.execution.BaseTestSqlTaskManager.MockDirectExchangeClientSupplier;
import io.trino.execution.buffer.OutputBuffers;
import io.trino.execution.scheduler.NodeScheduler;
import io.trino.execution.scheduler.NodeSchedulerConfig;
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java b/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java
index b57d796277dc..fc5a1434b130 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java
+++ b/core/trino-main/src/test/java/io/trino/execution/TestMemoryRevokingScheduler.java
@@ -24,6 +24,7 @@
import io.trino.exchange.ExchangeManagerRegistry;
import io.trino.execution.buffer.PipelinedOutputBuffers;
import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
import io.trino.memory.MemoryPool;
import io.trino.memory.QueryContext;
import io.trino.memory.context.LocalMemoryContext;
@@ -85,7 +86,7 @@ public void setUp()
{
memoryPool = new MemoryPool(DataSize.ofBytes(10));
- taskExecutor = new TaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
+ taskExecutor = new TimeSharingTaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
taskExecutor.start();
// Must be single threaded
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java
index 7ce9630da8c1..f47f8c6f4410 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java
+++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTask.java
@@ -31,6 +31,7 @@
import io.trino.execution.buffer.PipelinedOutputBuffers;
import io.trino.execution.buffer.PipelinedOutputBuffers.OutputBufferId;
import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
import io.trino.memory.MemoryPool;
import io.trino.memory.QueryContext;
import io.trino.operator.TaskContext;
@@ -96,7 +97,7 @@ public class TestSqlTask
@BeforeClass
public void setUp()
{
- taskExecutor = new TaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
+ taskExecutor = new TimeSharingTaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
taskExecutor.start();
taskNotificationExecutor = newScheduledThreadPool(10, threadsNamed("task-notification-%s"));
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java
index aedcb033b15b..6ec7a383fcd2 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java
+++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskExecution.java
@@ -34,6 +34,7 @@
import io.trino.execution.buffer.PipelinedOutputBuffers;
import io.trino.execution.buffer.PipelinedOutputBuffers.OutputBufferId;
import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
import io.trino.memory.MemoryPool;
import io.trino.memory.QueryContext;
import io.trino.memory.context.SimpleLocalMemoryContext;
@@ -101,7 +102,7 @@ public void testSimple()
{
ScheduledExecutorService taskNotificationExecutor = newScheduledThreadPool(10, threadsNamed("task-notification-%s"));
ScheduledExecutorService driverYieldExecutor = newScheduledThreadPool(2, threadsNamed("driver-yield-%s"));
- TaskExecutor taskExecutor = new TaskExecutor(5, 10, 3, 4, Ticker.systemTicker());
+ TaskExecutor taskExecutor = new TimeSharingTaskExecutor(5, 10, 3, 4, Ticker.systemTicker());
taskExecutor.start();
try {
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java
new file mode 100644
index 000000000000..bbec2769cdc5
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerThreadPerDriver.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution;
+
+import com.google.common.base.Ticker;
+import io.airlift.tracing.Tracing;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.dedicated.ThreadPerDriverTaskExecutor;
+import io.trino.execution.executor.scheduler.FairScheduler;
+
+import static io.trino.version.EmbedVersion.testingVersionEmbedder;
+
+public class TestSqlTaskManagerThreadPerDriver
+ extends BaseTestSqlTaskManager
+{
+ @Override
+ protected TaskExecutor createTaskExecutor()
+ {
+ return new ThreadPerDriverTaskExecutor(
+ Tracing.noopTracer(),
+ testingVersionEmbedder(),
+ new FairScheduler(8, "Runner-%d", Ticker.systemTicker()));
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerTimeSharing.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerTimeSharing.java
new file mode 100644
index 000000000000..aab4c2fdd704
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlTaskManagerTimeSharing.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution;
+
+import com.google.common.base.Ticker;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
+
+public class TestSqlTaskManagerTimeSharing
+ extends BaseTestSqlTaskManager
+{
+ @Override
+ protected TaskExecutor createTaskExecutor()
+ {
+ return new TimeSharingTaskExecutor(8, 16, 3, 4, Ticker.systemTicker());
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java b/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java
new file mode 100644
index 000000000000..09fc44900410
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/TestTaskExecutorStuckSplits.java
@@ -0,0 +1,262 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
+import io.airlift.node.NodeInfo;
+import io.airlift.stats.TestingGcMonitor;
+import io.airlift.testing.TestingTicker;
+import io.airlift.units.Duration;
+import io.opentelemetry.api.trace.Span;
+import io.trino.Session;
+import io.trino.connector.CatalogProperties;
+import io.trino.connector.ConnectorServices;
+import io.trino.connector.ConnectorServicesProvider;
+import io.trino.exchange.ExchangeManagerRegistry;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.TaskHandle;
+import io.trino.execution.executor.timesharing.TimeSharingTaskExecutor;
+import io.trino.memory.LocalMemoryManager;
+import io.trino.memory.NodeMemoryConfig;
+import io.trino.spi.connector.CatalogHandle;
+import io.trino.spiller.LocalSpillManager;
+import io.trino.spiller.NodeSpillConfig;
+import io.trino.version.EmbedVersion;
+import org.testng.annotations.Test;
+
+import java.util.List;
+import java.util.OptionalInt;
+import java.util.Set;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeoutException;
+import java.util.function.Predicate;
+
+import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
+import static io.airlift.tracing.Tracing.noopTracer;
+import static io.trino.execution.TaskTestUtils.createTestSplitMonitor;
+import static io.trino.execution.TaskTestUtils.createTestingPlanner;
+import static java.util.concurrent.TimeUnit.SECONDS;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertTrue;
+
+public class TestTaskExecutorStuckSplits
+{
+ @Test
+ public void testFailStuckSplitTasks()
+ throws InterruptedException, ExecutionException, TimeoutException
+ {
+ TestingTicker ticker = new TestingTicker();
+ TaskManagementExecutor taskManagementExecutor = new TaskManagementExecutor();
+
+ TaskId taskId = new TaskId(new StageId("query", 0), 1, 0);
+
+ TaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 8, 3, 4, ticker);
+ TaskHandle taskHandle = taskExecutor.addTask(
+ taskId,
+ () -> 1.0,
+ 1,
+ new Duration(1, SECONDS),
+ OptionalInt.of(1));
+
+ // Here we explicitly enqueue an indefinite running split runner
+ MockSplitRunner mockSplitRunner = new MockSplitRunner();
+ taskExecutor.enqueueSplits(taskHandle, false, ImmutableList.of(mockSplitRunner));
+
+ taskExecutor.start();
+ try {
+ // wait for the task executor to start processing the split
+ mockSplitRunner.waitForStart();
+
+ TaskManagerConfig taskManagerConfig = new TaskManagerConfig()
+ .setInterruptStuckSplitTasksEnabled(true)
+ .setInterruptStuckSplitTasksDetectionInterval(new Duration(10, SECONDS))
+ .setInterruptStuckSplitTasksWarningThreshold(new Duration(10, SECONDS))
+ .setInterruptStuckSplitTasksTimeout(new Duration(10, SECONDS));
+
+ try (SqlTaskManager sqlTaskManager = createSqlTaskManager(taskManagerConfig, new NodeMemoryConfig(), taskExecutor, taskManagementExecutor, stackTraceElements -> true)) {
+ sqlTaskManager.addStateChangeListener(taskId, (state) -> {
+ if (state.isTerminatingOrDone() && !taskHandle.isDestroyed()) {
+ taskExecutor.removeTask(taskHandle);
+ }
+ });
+
+ ticker.increment(30, SECONDS);
+ sqlTaskManager.failStuckSplitTasks();
+
+ mockSplitRunner.waitForFinish();
+ List taskInfos = sqlTaskManager.getAllTaskInfo();
+ assertEquals(taskInfos.size(), 1);
+
+ TaskInfo taskInfo = pollTerminatingTaskInfoUntilDone(sqlTaskManager, taskInfos.get(0));
+ assertEquals(taskInfo.getTaskStatus().getState(), TaskState.FAILED);
+ }
+ }
+ finally {
+ taskExecutor.stop();
+ taskManagementExecutor.close();
+ }
+ }
+
+ private SqlTaskManager createSqlTaskManager(
+ TaskManagerConfig taskManagerConfig,
+ NodeMemoryConfig nodeMemoryConfig,
+ TaskExecutor taskExecutor,
+ TaskManagementExecutor taskManagementExecutor,
+ Predicate> stuckSplitStackTracePredicate)
+ {
+ return new SqlTaskManager(
+ new EmbedVersion("testversion"),
+ new NoConnectorServicesProvider(),
+ createTestingPlanner(),
+ new BaseTestSqlTaskManager.MockLocationFactory(),
+ taskExecutor,
+ createTestSplitMonitor(),
+ new NodeInfo("test"),
+ new LocalMemoryManager(new NodeMemoryConfig()),
+ taskManagementExecutor,
+ taskManagerConfig,
+ nodeMemoryConfig,
+ new LocalSpillManager(new NodeSpillConfig()),
+ new NodeSpillConfig(),
+ new TestingGcMonitor(),
+ noopTracer(),
+ new ExchangeManagerRegistry(),
+ stuckSplitStackTracePredicate);
+ }
+
+ private static TaskInfo pollTerminatingTaskInfoUntilDone(SqlTaskManager taskManager, TaskInfo taskInfo)
+ throws InterruptedException, ExecutionException, TimeoutException
+ {
+ assertTrue(taskInfo.getTaskStatus().getState().isTerminatingOrDone());
+ int attempts = 3;
+ while (attempts > 0 && taskInfo.getTaskStatus().getState().isTerminating()) {
+ taskInfo = taskManager.getTaskInfo(taskInfo.getTaskStatus().getTaskId(), taskInfo.getTaskStatus().getVersion()).get(5, SECONDS);
+ attempts--;
+ }
+ return taskInfo;
+ }
+
+ private static class NoConnectorServicesProvider
+ implements ConnectorServicesProvider
+ {
+ @Override
+ public void loadInitialCatalogs() {}
+
+ @Override
+ public void ensureCatalogsLoaded(Session session, List catalogs) {}
+
+ @Override
+ public void pruneCatalogs(Set catalogsInUse)
+ {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public ConnectorServices getConnectorServices(CatalogHandle catalogHandle)
+ {
+ throw new UnsupportedOperationException();
+ }
+ }
+
+ private static class MockSplitRunner
+ implements SplitRunner
+ {
+ private final SettableFuture startedFuture = SettableFuture.create();
+ private final SettableFuture finishedFuture = SettableFuture.create();
+
+ @GuardedBy("this")
+ private Thread runnerThread;
+ @GuardedBy("this")
+ private boolean closed;
+
+ public void waitForStart()
+ throws ExecutionException, InterruptedException, TimeoutException
+ {
+ startedFuture.get(10, SECONDS);
+ }
+
+ public void waitForFinish()
+ throws ExecutionException, InterruptedException, TimeoutException
+ {
+ finishedFuture.get(10, SECONDS);
+ }
+
+ @Override
+ public int getPipelineId()
+ {
+ return 0;
+ }
+
+ @Override
+ public Span getPipelineSpan()
+ {
+ return Span.getInvalid();
+ }
+
+ @Override
+ public synchronized boolean isFinished()
+ {
+ return closed;
+ }
+
+ @Override
+ public ListenableFuture processFor(Duration duration)
+ {
+ startedFuture.set(null);
+ synchronized (this) {
+ runnerThread = Thread.currentThread();
+
+ if (closed) {
+ finishedFuture.set(null);
+ return immediateVoidFuture();
+ }
+ }
+
+ while (true) {
+ try {
+ Thread.sleep(100000);
+ }
+ catch (InterruptedException e) {
+ break;
+ }
+ }
+
+ synchronized (this) {
+ closed = true;
+ }
+ finishedFuture.set(null);
+
+ return immediateVoidFuture();
+ }
+
+ @Override
+ public String getInfo()
+ {
+ return "MockSplitRunner";
+ }
+
+ @Override
+ public synchronized void close()
+ {
+ closed = true;
+
+ if (runnerThread != null) {
+ runnerThread.interrupt();
+ }
+ }
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java b/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java
index ed74d9203560..0030d5d88fe2 100644
--- a/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java
+++ b/core/trino-main/src/test/java/io/trino/execution/TestTaskManagerConfig.java
@@ -40,6 +40,7 @@ public class TestTaskManagerConfig
public void testDefaults()
{
assertRecordedDefaults(recordDefaults(TaskManagerConfig.class)
+ .setThreadPerDriverSchedulerEnabled(false)
.setInitialSplitsPerNode(Runtime.getRuntime().availableProcessors() * 2)
.setSplitConcurrencyAdjustmentInterval(new Duration(100, TimeUnit.MILLISECONDS))
.setStatusRefreshMaxWait(new Duration(1, TimeUnit.SECONDS))
@@ -85,6 +86,7 @@ public void testExplicitPropertyMappings()
int processorCount = DEFAULT_PROCESSOR_COUNT == 32 ? 16 : 32;
int maxWriterCount = DEFAULT_SCALE_WRITERS_MAX_WRITER_COUNT == 32 ? 16 : 32;
Map properties = ImmutableMap.builder()
+ .put("experimental.thread-per-driver-scheduler-enabled", "true")
.put("task.initial-splits-per-node", "1")
.put("task.split-concurrency-adjustment-interval", "1s")
.put("task.status-refresh-max-wait", "2s")
@@ -125,6 +127,7 @@ public void testExplicitPropertyMappings()
.buildOrThrow();
TaskManagerConfig expected = new TaskManagerConfig()
+ .setThreadPerDriverSchedulerEnabled(true)
.setInitialSplitsPerNode(1)
.setSplitConcurrencyAdjustmentInterval(new Duration(1, TimeUnit.SECONDS))
.setStatusRefreshMaxWait(new Duration(2, TimeUnit.SECONDS))
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java b/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java
new file mode 100644
index 000000000000..2648bfbc581b
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/dedicated/TestThreadPerDriverTaskExecutor.java
@@ -0,0 +1,258 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.dedicated;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.airlift.testing.TestingTicker;
+import io.airlift.units.Duration;
+import io.opentelemetry.api.trace.Span;
+import io.trino.execution.SplitRunner;
+import io.trino.execution.StageId;
+import io.trino.execution.TaskId;
+import io.trino.execution.TaskManagerConfig;
+import io.trino.execution.executor.TaskHandle;
+import io.trino.execution.executor.scheduler.FairScheduler;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.List;
+import java.util.OptionalInt;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Phaser;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+
+import static io.airlift.tracing.Tracing.noopTracer;
+import static io.trino.version.EmbedVersion.testingVersionEmbedder;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestThreadPerDriverTaskExecutor
+{
+ @Test
+ @Timeout(10)
+ public void testCancellationWhileProcessing()
+ throws ExecutionException, InterruptedException
+ {
+ ThreadPerDriverTaskExecutor executor = new ThreadPerDriverTaskExecutor(new TaskManagerConfig(), noopTracer(), testingVersionEmbedder());
+ executor.start();
+ try {
+ TaskId taskId = new TaskId(new StageId("query", 1), 1, 1);
+ TaskHandle task = executor.addTask(taskId, () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
+
+ CountDownLatch started = new CountDownLatch(1);
+
+ SplitRunner split = new TestingSplitRunner(ImmutableList.of(duration -> {
+ started.countDown();
+ try {
+ Thread.currentThread().join();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+
+ return Futures.immediateVoidFuture();
+ }));
+
+ ListenableFuture splitDone = executor.enqueueSplits(task, false, ImmutableList.of(split)).get(0);
+
+ started.await();
+ executor.removeTask(task);
+
+ splitDone.get();
+ assertThat(split.isFinished()).isTrue();
+ }
+ finally {
+ executor.stop();
+ }
+ }
+
+ @Test
+ @Timeout(10)
+ public void testBlocking()
+ throws ExecutionException, InterruptedException
+ {
+ ThreadPerDriverTaskExecutor executor = new ThreadPerDriverTaskExecutor(new TaskManagerConfig(), noopTracer(), testingVersionEmbedder());
+ executor.start();
+
+ try {
+ TaskId taskId = new TaskId(new StageId("query", 1), 1, 1);
+ TaskHandle task = executor.addTask(taskId, () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
+
+ TestFuture blocked = new TestFuture();
+
+ SplitRunner split = new TestingSplitRunner(ImmutableList.of(
+ duration -> blocked,
+ duration -> Futures.immediateVoidFuture()));
+
+ ListenableFuture splitDone = executor.enqueueSplits(task, false, ImmutableList.of(split)).get(0);
+
+ blocked.awaitListenerAdded();
+ blocked.set(null); // unblock the split
+
+ splitDone.get();
+ assertThat(split.isFinished()).isTrue();
+ }
+ finally {
+ executor.stop();
+ }
+ }
+
+ @Test
+ @Timeout(10)
+ public void testYielding()
+ throws ExecutionException, InterruptedException
+ {
+ TestingTicker ticker = new TestingTicker();
+ FairScheduler scheduler = new FairScheduler(1, "Runner-%d", ticker);
+ ThreadPerDriverTaskExecutor executor = new ThreadPerDriverTaskExecutor(noopTracer(), testingVersionEmbedder(), scheduler);
+ executor.start();
+
+ try {
+ TaskId taskId = new TaskId(new StageId("query", 1), 1, 1);
+ TaskHandle task = executor.addTask(taskId, () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
+
+ Phaser phaser = new Phaser(2);
+ SplitRunner split = new TestingSplitRunner(ImmutableList.of(
+ duration -> {
+ phaser.arriveAndAwaitAdvance(); // wait to start
+ phaser.arriveAndAwaitAdvance(); // wait to advance time
+ return Futures.immediateVoidFuture();
+ },
+ duration -> {
+ phaser.arriveAndAwaitAdvance();
+ return Futures.immediateVoidFuture();
+ }));
+
+ ListenableFuture splitDone = executor.enqueueSplits(task, false, ImmutableList.of(split)).get(0);
+
+ phaser.arriveAndAwaitAdvance(); // wait for split to start
+
+ // cause the task to yield
+ ticker.increment(FairScheduler.QUANTUM_NANOS * 2, TimeUnit.NANOSECONDS);
+ phaser.arriveAndAwaitAdvance();
+
+ // wait for reschedule
+ assertThat(phaser.arriveAndAwaitAdvance()).isEqualTo(3); // wait for reschedule
+
+ splitDone.get();
+ assertThat(split.isFinished()).isTrue();
+ }
+ finally {
+ executor.stop();
+ }
+ }
+
+ private static class TestFuture
+ extends AbstractFuture
+ {
+ private final CountDownLatch listenerAdded = new CountDownLatch(1);
+
+ @Override
+ public void addListener(Runnable listener, Executor executor)
+ {
+ super.addListener(listener, executor);
+ listenerAdded.countDown();
+ }
+
+ @Override
+ public boolean set(Void value)
+ {
+ return super.set(value);
+ }
+
+ public void awaitListenerAdded()
+ throws InterruptedException
+ {
+ listenerAdded.await();
+ }
+ }
+
+ private static class TestingSplitRunner
+ implements SplitRunner
+ {
+ private final List>> invocations;
+ private int invocation;
+ private volatile boolean finished;
+ private volatile Thread runnerThread;
+
+ public TestingSplitRunner(List>> invocations)
+ {
+ this.invocations = invocations;
+ }
+
+ @Override
+ public final int getPipelineId()
+ {
+ return 0;
+ }
+
+ @Override
+ public final Span getPipelineSpan()
+ {
+ return Span.getInvalid();
+ }
+
+ @Override
+ public final boolean isFinished()
+ {
+ return finished;
+ }
+
+ @Override
+ public final ListenableFuture processFor(Duration duration)
+ {
+ ListenableFuture blocked;
+
+ runnerThread = Thread.currentThread();
+ try {
+ blocked = invocations.get(invocation).apply(duration);
+ }
+ finally {
+ runnerThread = null;
+ }
+
+ invocation++;
+
+ if (invocation == invocations.size()) {
+ finished = true;
+ }
+
+ return blocked;
+ }
+
+ @Override
+ public final String getInfo()
+ {
+ return "";
+ }
+
+ @Override
+ public final void close()
+ {
+ finished = true;
+
+ Thread runnerThread = this.runnerThread;
+
+ if (runnerThread != null) {
+ runnerThread.interrupt();
+ }
+ }
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestFairScheduler.java b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestFairScheduler.java
new file mode 100644
index 000000000000..24dd6809159e
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestFairScheduler.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.SettableFuture;
+import io.airlift.testing.TestingTicker;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.Timeout;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestFairScheduler
+{
+ @Test
+ public void testBasic()
+ throws ExecutionException, InterruptedException
+ {
+ try (FairScheduler scheduler = FairScheduler.newInstance(1)) {
+ Group group = scheduler.createGroup("G1");
+
+ AtomicBoolean ran = new AtomicBoolean();
+ ListenableFuture done = scheduler.submit(group, 1, context -> ran.set(true));
+
+ done.get();
+ assertThat(ran.get())
+ .describedAs("Ran task")
+ .isTrue();
+ }
+ }
+
+ @Test
+ @Timeout(5)
+ public void testYield()
+ throws ExecutionException, InterruptedException
+ {
+ TestingTicker ticker = new TestingTicker();
+ try (FairScheduler scheduler = FairScheduler.newInstance(1, ticker)) {
+ Group group = scheduler.createGroup("G");
+
+ CountDownLatch task1Started = new CountDownLatch(1);
+ AtomicBoolean task2Ran = new AtomicBoolean();
+
+ ListenableFuture task1 = scheduler.submit(group, 1, context -> {
+ task1Started.countDown();
+ while (!task2Ran.get()) {
+ if (!context.maybeYield()) {
+ return;
+ }
+ }
+ });
+
+ task1Started.await();
+
+ ListenableFuture task2 = scheduler.submit(group, 2, context -> {
+ task2Ran.set(true);
+ });
+
+ while (!task2.isDone()) {
+ ticker.increment(FairScheduler.QUANTUM_NANOS * 2, TimeUnit.NANOSECONDS);
+ }
+
+ task1.get();
+ }
+ }
+
+ @Test
+ public void testBlocking()
+ throws InterruptedException, ExecutionException
+ {
+ try (FairScheduler scheduler = FairScheduler.newInstance(1)) {
+ Group group = scheduler.createGroup("G");
+
+ CountDownLatch task1Started = new CountDownLatch(1);
+ CountDownLatch task2Submitted = new CountDownLatch(1);
+ CountDownLatch task2Started = new CountDownLatch(1);
+ AtomicBoolean task2Ran = new AtomicBoolean();
+
+ SettableFuture task1Blocked = SettableFuture.create();
+
+ ListenableFuture task1 = scheduler.submit(group, 1, context -> {
+ try {
+ task1Started.countDown();
+ task2Submitted.await();
+
+ assertThat(task2Ran.get())
+ .describedAs("Task 2 run")
+ .isFalse();
+
+ context.block(task1Blocked);
+
+ assertThat(task2Ran.get())
+ .describedAs("Task 2 run")
+ .isTrue();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ });
+
+ task1Started.await();
+
+ ListenableFuture task2 = scheduler.submit(group, 2, context -> {
+ task2Started.countDown();
+ task2Ran.set(true);
+ });
+
+ task2Submitted.countDown();
+ task2Started.await();
+
+ // unblock task 1
+ task1Blocked.set(null);
+
+ task1.get();
+ task2.get();
+ }
+ }
+
+ @Test
+ public void testCancelWhileYielding()
+ throws InterruptedException, ExecutionException
+ {
+ TestingTicker ticker = new TestingTicker();
+ try (FairScheduler scheduler = FairScheduler.newInstance(1, ticker)) {
+ Group group = scheduler.createGroup("G");
+
+ CountDownLatch task1Started = new CountDownLatch(1);
+ CountDownLatch task1TimeAdvanced = new CountDownLatch(1);
+
+ ListenableFuture task1 = scheduler.submit(group, 1, context -> {
+ try {
+ task1Started.countDown();
+ task1TimeAdvanced.await();
+
+ assertThat(context.maybeYield())
+ .describedAs("Cancelled while yielding")
+ .isFalse();
+ }
+ catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ });
+
+ task1Started.await();
+ scheduler.pause(); // prevent rescheduling after yield
+
+ ticker.increment(FairScheduler.QUANTUM_NANOS * 2, TimeUnit.NANOSECONDS);
+ task1TimeAdvanced.countDown();
+
+ scheduler.removeGroup(group);
+ task1.get();
+ }
+ }
+
+ @Test
+ public void testCancelWhileBlocking()
+ throws InterruptedException, ExecutionException
+ {
+ TestingTicker ticker = new TestingTicker();
+ try (FairScheduler scheduler = FairScheduler.newInstance(1, ticker)) {
+ Group group = scheduler.createGroup("G");
+
+ CountDownLatch task1Started = new CountDownLatch(1);
+ TestFuture task1Blocked = new TestFuture();
+
+ ListenableFuture task1 = scheduler.submit(group, 1, context -> {
+ task1Started.countDown();
+
+ assertThat(context.block(task1Blocked))
+ .describedAs("Cancelled while blocking")
+ .isFalse();
+ });
+
+ task1Started.await();
+
+ task1Blocked.awaitListenerAdded(); // When the listener is added, we know the task is blocked
+
+ scheduler.removeGroup(group);
+ task1.get();
+ }
+ }
+
+ private static class TestFuture
+ extends AbstractFuture
+ {
+ private final CountDownLatch listenerAdded = new CountDownLatch(1);
+
+ @Override
+ public void addListener(Runnable listener, Executor executor)
+ {
+ super.addListener(listener, executor);
+ listenerAdded.countDown();
+ }
+
+ @Override
+ public boolean set(Void value)
+ {
+ return super.set(value);
+ }
+
+ public void awaitListenerAdded()
+ throws InterruptedException
+ {
+ listenerAdded.await();
+ }
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestPriorityQueue.java b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestPriorityQueue.java
new file mode 100644
index 000000000000..d4f799e1a5e3
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestPriorityQueue.java
@@ -0,0 +1,210 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import com.google.common.collect.ImmutableSet;
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+public class TestPriorityQueue
+{
+ @Test
+ public void testEmpty()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ assertThat(queue.poll()).isNull();
+ assertThat(queue.isEmpty()).isTrue();
+ }
+
+ @Test
+ public void testNotEmpty()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("hello", 1);
+ assertThat(queue.isEmpty()).isFalse();
+ }
+
+ @Test
+ public void testDuplicate()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("hello", 1);
+ assertThatThrownBy(() -> queue.add("hello", 2))
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
+ @Test
+ public void testOrder()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("jumps", 5);
+ queue.add("fox", 4);
+ queue.add("over", 6);
+ queue.add("brown", 3);
+ queue.add("dog", 8);
+ queue.add("the", 1);
+ queue.add("lazy", 7);
+ queue.add("quick", 2);
+
+ assertThat(queue.poll()).isEqualTo("the");
+ assertThat(queue.poll()).isEqualTo("quick");
+ assertThat(queue.poll()).isEqualTo("brown");
+ assertThat(queue.poll()).isEqualTo("fox");
+ assertThat(queue.poll()).isEqualTo("jumps");
+ assertThat(queue.poll()).isEqualTo("over");
+ assertThat(queue.poll()).isEqualTo("lazy");
+ assertThat(queue.poll()).isEqualTo("dog");
+ assertThat(queue.poll()).isNull();
+ }
+
+ @Test
+ public void testInterleaved()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("jumps", 5);
+ queue.add("over", 6);
+ queue.add("fox", 4);
+
+ assertThat(queue.poll()).isEqualTo("fox");
+ assertThat(queue.poll()).isEqualTo("jumps");
+
+ queue.add("brown", 3);
+ queue.add("dog", 8);
+ queue.add("the", 1);
+
+ assertThat(queue.poll()).isEqualTo("the");
+ assertThat(queue.poll()).isEqualTo("brown");
+ assertThat(queue.poll()).isEqualTo("over");
+
+ queue.add("lazy", 7);
+ queue.add("quick", 2);
+
+ assertThat(queue.poll()).isEqualTo("quick");
+ assertThat(queue.poll()).isEqualTo("lazy");
+ assertThat(queue.poll()).isEqualTo("dog");
+ assertThat(queue.poll()).isNull();
+ }
+
+ @Test
+ public void testRemove()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("fox", 4);
+ queue.add("brown", 3);
+ queue.add("the", 1);
+ queue.add("quick", 2);
+
+ queue.remove("brown");
+
+ assertThat(queue.poll()).isEqualTo("the");
+ assertThat(queue.poll()).isEqualTo("quick");
+ assertThat(queue.poll()).isEqualTo("fox");
+ assertThat(queue.poll()).isNull();
+ }
+
+ @Test
+ public void testRemoveMissing()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("the", 1);
+ queue.add("quick", 2);
+ queue.add("brown", 3);
+
+ assertThatThrownBy(() -> queue.remove("fox"))
+ .isInstanceOf(IllegalArgumentException.class);
+
+ queue.removeIfPresent("fox");
+ }
+
+ @Test
+ public void testContains()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("the", 1);
+ queue.add("quick", 2);
+ queue.add("brown", 3);
+
+ assertThat(queue.contains("quick")).isTrue();
+ assertThat(queue.contains("fox")).isFalse();
+ }
+
+ @Test
+ public void testRecycle()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("hello", 1);
+ assertThat(queue.poll()).isEqualTo("hello");
+
+ queue.add("hello", 2);
+ assertThat(queue.poll()).isEqualTo("hello");
+ }
+
+ @Test
+ public void testValues()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ assertThat(queue.values()).isEmpty();
+
+ queue.add("hello", 1);
+ queue.add("world", 2);
+
+ assertThat(queue.values())
+ .isEqualTo(ImmutableSet.of("hello", "world"));
+ }
+
+ @Test
+ public void testNextPriority()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ assertThatThrownBy(queue::nextPriority)
+ .isInstanceOf(IllegalStateException.class);
+
+ queue.add("hello", 10);
+ queue.add("world", 20);
+
+ assertThat(queue.nextPriority()).isEqualTo(10);
+
+ queue.poll();
+ assertThat(queue.nextPriority()).isEqualTo(20);
+
+ queue.poll();
+ assertThatThrownBy(queue::nextPriority)
+ .isInstanceOf(IllegalStateException.class);
+ }
+
+ @Test
+ public void testSamePriority()
+ {
+ PriorityQueue queue = new PriorityQueue<>();
+
+ queue.add("hello", 1);
+ queue.add("world", 1);
+
+ assertThat(queue.poll()).isEqualTo("hello");
+ assertThat(queue.poll()).isEqualTo("world");
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestSchedulingQueue.java b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestSchedulingQueue.java
new file mode 100644
index 000000000000..cc31c9db2d86
--- /dev/null
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/scheduler/TestSchedulingQueue.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package io.trino.execution.executor.scheduler;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+public class TestSchedulingQueue
+{
+ @Test
+ public void testEmpty()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ assertThat(queue.dequeue(1)).isNull();
+ }
+
+ @Test
+ public void testSingleGroup()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+
+ queue.enqueue("G1", "T1", 1);
+ queue.enqueue("G1", "T2", 3);
+ queue.enqueue("G1", "T3", 5);
+ queue.enqueue("G1", "T4", 7);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1");
+ assertThat(queue.dequeue(1)).isEqualTo("T2");
+ assertThat(queue.dequeue(1)).isEqualTo("T3");
+ assertThat(queue.dequeue(1)).isEqualTo("T4");
+
+ queue.enqueue("G1", "T1", 10);
+ queue.enqueue("G1", "T2", 10);
+ queue.enqueue("G1", "T3", 10);
+ queue.enqueue("G1", "T4", 10);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1");
+ assertThat(queue.dequeue(1)).isEqualTo("T2");
+ assertThat(queue.dequeue(1)).isEqualTo("T3");
+ assertThat(queue.dequeue(1)).isEqualTo("T4");
+
+ queue.enqueue("G1", "T1", 16);
+ queue.enqueue("G1", "T2", 12);
+ queue.enqueue("G1", "T3", 8);
+ queue.enqueue("G1", "T4", 4);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T4");
+ assertThat(queue.dequeue(1)).isEqualTo("T3");
+ assertThat(queue.dequeue(1)).isEqualTo("T2");
+ assertThat(queue.dequeue(1)).isEqualTo("T1");
+
+ queue.finish("G1", "T1");
+ queue.finish("G1", "T2");
+ queue.finish("G1", "T3");
+ queue.finish("G1", "T4");
+
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+ }
+
+ @Test
+ public void testBasic()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.startGroup("G2");
+
+ queue.enqueue("G1", "T1.0", 1);
+ queue.enqueue("G1", "T1.1", 2);
+ queue.enqueue("G2", "T2.0", 3);
+ queue.enqueue("G2", "T2.1", 4);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T1.1");
+ assertThat(queue.dequeue(1)).isEqualTo("T2.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T2.1");
+
+ queue.enqueue("G1", "T1.0", 10);
+ queue.enqueue("G1", "T1.1", 20);
+ queue.enqueue("G2", "T2.0", 15);
+ queue.enqueue("G2", "T2.1", 5);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T2.1");
+ assertThat(queue.dequeue(1)).isEqualTo("T2.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T1.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T1.1");
+
+ queue.enqueue("G1", "T1.0", 100);
+ queue.enqueue("G2", "T2.0", 90);
+ assertThat(queue.dequeue(1)).isEqualTo("T2.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T1.0");
+ }
+
+ @Test
+ public void testSomeEmptyGroups()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.startGroup("G2");
+
+ queue.enqueue("G2", "T1", 0);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1");
+ }
+
+ @Test
+ public void testDelayedCreation()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.startGroup("G2");
+
+ queue.enqueue("G1", "T1.0", 100);
+ queue.enqueue("G2", "T2.0", 200);
+
+ queue.startGroup("G3"); // new group gets a priority baseline equal to the minimum current priority
+ queue.enqueue("G3", "T3.0", 50);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T3.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T2.0");
+ }
+
+ @Test
+ public void testDelayedCreationWhileAllRunning()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.startGroup("G2");
+
+ queue.enqueue("G1", "T1.0", 0);
+
+ queue.enqueue("G2", "T2.0", 100);
+ queue.dequeue(50);
+ queue.dequeue(50);
+
+ queue.startGroup("G3"); // new group gets a priority baseline equal to the minimum current priority
+ queue.enqueue("G3", "T3.0", 10);
+
+ queue.enqueue("G1", "T1.0", 50);
+ queue.enqueue("G2", "T2.0", 50);
+
+ assertThat(queue.dequeue(1)).isEqualTo("T1.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T3.0");
+ assertThat(queue.dequeue(1)).isEqualTo("T2.0");
+ }
+
+ @Test
+ public void testGroupState()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ // initial state with no tasks
+ queue.startGroup("G1");
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+
+ // after adding a task, it should be runnable
+ queue.enqueue("G1", "T1", 0);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+ queue.enqueue("G1", "T2", 0);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+
+ // after dequeueing, still runnable if there's at least one runnable task
+ queue.dequeue(1);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+
+ // after all tasks are dequeued, it should be running
+ queue.dequeue(1);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNING);
+
+ // still running while at least one task is running and there are no runnable tasks
+ queue.block("G1", "T1", 1);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNING);
+
+ // runnable after blocking when there are still runnable tasks
+ queue.enqueue("G1", "T1", 1);
+ queue.block("G1", "T2", 1);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+
+ // blocked when all tasks are blocked
+ queue.dequeue(1);
+ queue.block("G1", "T1", 1);
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+
+ // back to runnable after unblocking
+ queue.enqueue("G1", "T1", 1);
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+ }
+
+ @Test
+ public void testNonGreedyDeque()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.startGroup("G2");
+
+ queue.enqueue("G1", "T1.0", 0);
+ queue.enqueue("G2", "T2.0", 1);
+
+ queue.enqueue("G1", "T1.1", 2);
+ queue.enqueue("G1", "T1.2", 3);
+
+ queue.enqueue("G2", "T2.1", 2);
+ queue.enqueue("G2", "T2.2", 3);
+
+ assertThat(queue.dequeue(2)).isEqualTo("T1.0");
+ assertThat(queue.dequeue(2)).isEqualTo("T2.0");
+ assertThat(queue.dequeue(2)).isEqualTo("T1.1");
+ assertThat(queue.dequeue(2)).isEqualTo("T2.1");
+ assertThat(queue.dequeue(2)).isEqualTo("T1.2");
+ assertThat(queue.dequeue(2)).isEqualTo("T2.2");
+ assertThat(queue.dequeue(2)).isNull();
+ }
+
+ @Test
+ public void testFinishTask()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.enqueue("G1", "T1", 0);
+ queue.enqueue("G1", "T2", 1);
+ queue.enqueue("G1", "T3", 2);
+
+ assertThat(queue.peek()).isEqualTo("T1");
+ queue.finish("G1", "T1");
+ assertThat(queue.peek()).isEqualTo("T2");
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+
+ // check that the group becomes not-runnable
+ queue.finish("G1", "T2");
+ queue.finish("G1", "T3");
+ assertThat(queue.peek()).isNull();
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+
+ // check that the group becomes runnable again
+ queue.enqueue("G1", "T4", 0);
+ assertThat(queue.peek()).isEqualTo("T4");
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+ }
+
+ @Test
+ public void testFinishTaskWhileRunning()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+ queue.startGroup("G1");
+
+ queue.enqueue("G1", "T1", 0);
+ queue.enqueue("G1", "T2", 1);
+ queue.enqueue("G1", "T3", 2);
+ assertThat(queue.dequeue(0)).isEqualTo("T1");
+ assertThat(queue.dequeue(0)).isEqualTo("T2");
+ assertThat(queue.peek()).isEqualTo("T3");
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNABLE);
+
+ queue.finish("G1", "T3");
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNING);
+
+ queue.finish("G1", "T1");
+ assertThat(queue.state("G1")).isEqualTo(State.RUNNING);
+
+ queue.finish("G1", "T2");
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+ }
+
+ @Test
+ public void testFinishTaskWhileBlocked()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+ queue.startGroup("G1");
+
+ queue.enqueue("G1", "T1", 0);
+ queue.enqueue("G1", "T2", 1);
+ assertThat(queue.dequeue(0)).isEqualTo("T1");
+ assertThat(queue.dequeue(0)).isEqualTo("T2");
+ queue.block("G1", "T1", 0);
+ queue.block("G1", "T2", 0);
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+
+ queue.finish("G1", "T1");
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+
+ queue.finish("G1", "T2");
+ assertThat(queue.state("G1")).isEqualTo(State.BLOCKED);
+ }
+
+ @Test
+ public void testFinishGroup()
+ {
+ SchedulingQueue queue = new SchedulingQueue<>();
+
+ queue.startGroup("G1");
+ queue.enqueue("G1", "T1.1", 0);
+ assertThat(queue.peek()).isEqualTo("T1.1");
+
+ queue.startGroup("G2");
+ queue.enqueue("G2", "T2.1", 1);
+ assertThat(queue.peek()).isEqualTo("T1.1");
+
+ queue.finishGroup("G1");
+ assertThat(queue.containsGroup("G1")).isFalse();
+ assertThat(queue.peek()).isEqualTo("T2.1");
+ }
+}
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/Histogram.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/Histogram.java
similarity index 99%
rename from core/trino-main/src/test/java/io/trino/execution/executor/Histogram.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/Histogram.java
index 343f02c30607..8262c14d95aa 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/Histogram.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/Histogram.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.collect.ImmutableList;
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationController.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationController.java
similarity index 93%
rename from core/trino-main/src/test/java/io/trino/execution/executor/SimulationController.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationController.java
index d8771580b4da..990a351170e9 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationController.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationController.java
@@ -11,16 +11,16 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.collect.ArrayListMultimap;
import com.google.common.collect.ListMultimap;
import com.google.common.collect.Multimaps;
import io.trino.execution.StageId;
import io.trino.execution.TaskId;
-import io.trino.execution.executor.SimulationTask.IntermediateTask;
-import io.trino.execution.executor.SimulationTask.LeafTask;
-import io.trino.execution.executor.SplitGenerators.SplitGenerator;
+import io.trino.execution.executor.timesharing.SimulationTask.IntermediateTask;
+import io.trino.execution.executor.timesharing.SimulationTask.LeafTask;
+import io.trino.execution.executor.timesharing.SplitGenerators.SplitGenerator;
import java.util.Map;
import java.util.OptionalInt;
@@ -29,7 +29,7 @@
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.BiConsumer;
-import static io.trino.execution.executor.SimulationController.TaskSpecification.Type.LEAF;
+import static io.trino.execution.executor.timesharing.SimulationController.TaskSpecification.Type.LEAF;
import static java.util.concurrent.Executors.newSingleThreadExecutor;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -37,8 +37,8 @@ class SimulationController
{
private static final int DEFAULT_MIN_SPLITS_PER_TASK = 3;
- private final TaskExecutor taskExecutor;
- private final BiConsumer callback;
+ private final TimeSharingTaskExecutor taskExecutor;
+ private final BiConsumer callback;
private final ExecutorService controllerExecutor = newSingleThreadExecutor();
@@ -50,7 +50,7 @@ class SimulationController
private final AtomicBoolean stopped = new AtomicBoolean();
- public SimulationController(TaskExecutor taskExecutor, BiConsumer callback)
+ public SimulationController(TimeSharingTaskExecutor taskExecutor, BiConsumer callback)
{
this.taskExecutor = taskExecutor;
this.callback = callback;
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationSplit.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationSplit.java
similarity index 99%
rename from core/trino-main/src/test/java/io/trino/execution/executor/SimulationSplit.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationSplit.java
index 54ac4c2cd2b1..9cf452655c07 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationSplit.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationSplit.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.SettableFuture;
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationTask.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationTask.java
similarity index 86%
rename from core/trino-main/src/test/java/io/trino/execution/executor/SimulationTask.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationTask.java
index ed2ae7c1790c..e42314723244 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/SimulationTask.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SimulationTask.java
@@ -11,13 +11,14 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Sets;
import io.airlift.units.Duration;
import io.trino.execution.TaskId;
-import io.trino.execution.executor.SimulationController.TaskSpecification;
+import io.trino.execution.executor.TaskHandle;
+import io.trino.execution.executor.timesharing.SimulationController.TaskSpecification;
import java.util.OptionalInt;
import java.util.Set;
@@ -36,7 +37,7 @@ abstract class SimulationTask
private final TaskHandle taskHandle;
private final AtomicBoolean killed = new AtomicBoolean();
- public SimulationTask(TaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
+ public SimulationTask(TimeSharingTaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
{
this.specification = specification;
this.taskId = taskId;
@@ -123,21 +124,21 @@ public long getScheduledTimeNanos()
return runningWallTime + completedWallTime;
}
- public abstract void schedule(TaskExecutor taskExecutor, int numSplits);
+ public abstract void schedule(TimeSharingTaskExecutor taskExecutor, int numSplits);
public static class LeafTask
extends SimulationTask
{
private final TaskSpecification taskSpecification;
- public LeafTask(TaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
+ public LeafTask(TimeSharingTaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
{
super(taskExecutor, specification, taskId);
this.taskSpecification = specification;
}
@Override
- public void schedule(TaskExecutor taskExecutor, int numSplits)
+ public void schedule(TimeSharingTaskExecutor taskExecutor, int numSplits)
{
ImmutableList.Builder splits = ImmutableList.builder();
for (int i = 0; i < numSplits; i++) {
@@ -153,14 +154,14 @@ public static class IntermediateTask
{
private final SplitSpecification splitSpecification;
- public IntermediateTask(TaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
+ public IntermediateTask(TimeSharingTaskExecutor taskExecutor, TaskSpecification specification, TaskId taskId)
{
super(taskExecutor, specification, taskId);
this.splitSpecification = specification.nextSpecification();
}
@Override
- public void schedule(TaskExecutor taskExecutor, int numSplits)
+ public void schedule(TimeSharingTaskExecutor taskExecutor, int numSplits)
{
ImmutableList.Builder splits = ImmutableList.builderWithExpectedSize(numSplits);
for (int i = 0; i < numSplits; i++) {
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/SplitGenerators.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitGenerators.java
similarity index 97%
rename from core/trino-main/src/test/java/io/trino/execution/executor/SplitGenerators.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitGenerators.java
index 9a4aebdd0895..ebcd33f40450 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/SplitGenerators.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitGenerators.java
@@ -11,19 +11,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.collect.ImmutableList;
import io.airlift.units.Duration;
-import io.trino.execution.executor.SplitSpecification.IntermediateSplitSpecification;
-import io.trino.execution.executor.SplitSpecification.LeafSplitSpecification;
+import io.trino.execution.executor.timesharing.SplitSpecification.IntermediateSplitSpecification;
+import io.trino.execution.executor.timesharing.SplitSpecification.LeafSplitSpecification;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ThreadLocalRandom;
-import static io.trino.execution.executor.Histogram.fromContinuous;
+import static io.trino.execution.executor.timesharing.Histogram.fromContinuous;
import static java.util.concurrent.TimeUnit.DAYS;
import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/SplitSpecification.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitSpecification.java
similarity index 93%
rename from core/trino-main/src/test/java/io/trino/execution/executor/SplitSpecification.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitSpecification.java
index f998ea8f7a7c..f88b63a893f2 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/SplitSpecification.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/SplitSpecification.java
@@ -11,10 +11,10 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
-import io.trino.execution.executor.SimulationSplit.IntermediateSplit;
-import io.trino.execution.executor.SimulationSplit.LeafSplit;
+import io.trino.execution.executor.timesharing.SimulationSplit.IntermediateSplit;
+import io.trino.execution.executor.timesharing.SimulationSplit.LeafSplit;
import java.util.concurrent.ScheduledExecutorService;
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/TestTaskExecutor.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TestTimeSharingTaskExecutor.java
similarity index 92%
rename from core/trino-main/src/test/java/io/trino/execution/executor/TestTaskExecutor.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TestTimeSharingTaskExecutor.java
index 484b58231c44..1625000b2bd0 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/TestTaskExecutor.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TestTimeSharingTaskExecutor.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
@@ -23,6 +23,8 @@
import io.trino.execution.SplitRunner;
import io.trino.execution.StageId;
import io.trino.execution.TaskId;
+import io.trino.execution.executor.TaskExecutor;
+import io.trino.execution.executor.TaskHandle;
import io.trino.spi.QueryId;
import org.testng.annotations.Test;
@@ -39,8 +41,8 @@
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.airlift.testing.Assertions.assertGreaterThan;
import static io.airlift.testing.Assertions.assertLessThan;
-import static io.trino.execution.executor.MultilevelSplitQueue.LEVEL_CONTRIBUTION_CAP;
-import static io.trino.execution.executor.MultilevelSplitQueue.LEVEL_THRESHOLD_SECONDS;
+import static io.trino.execution.executor.timesharing.MultilevelSplitQueue.LEVEL_CONTRIBUTION_CAP;
+import static io.trino.execution.executor.timesharing.MultilevelSplitQueue.LEVEL_THRESHOLD_SECONDS;
import static java.lang.Double.isNaN;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static java.util.concurrent.TimeUnit.MINUTES;
@@ -49,7 +51,7 @@
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;
-public class TestTaskExecutor
+public class TestTimeSharingTaskExecutor
{
@Test(invocationCount = 100)
public void testTasksComplete()
@@ -58,7 +60,7 @@ public void testTasksComplete()
TestingTicker ticker = new TestingTicker();
Duration splitProcessingDurationThreshold = new Duration(10, MINUTES);
- TaskExecutor taskExecutor = new TaskExecutor(4, 8, 3, 4, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 8, 3, 4, ticker);
taskExecutor.start();
try {
@@ -156,7 +158,7 @@ public void testTasksComplete()
public void testQuantaFairness()
{
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(1, 2, 3, 4, ticker);
+ TaskExecutor taskExecutor = new TimeSharingTaskExecutor(1, 2, 3, 4, ticker);
taskExecutor.start();
try {
@@ -190,12 +192,12 @@ public void testQuantaFairness()
public void testLevelMovement()
{
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(2, 2, 3, 4, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(2, 2, 3, 4, ticker);
taskExecutor.start();
try {
ticker.increment(20, MILLISECONDS);
- TaskHandle testTaskHandle = taskExecutor.addTask(new TaskId(new StageId("test", 0), 0, 0), () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
+ TimeSharingTaskHandle testTaskHandle = taskExecutor.addTask(new TaskId(new StageId("test", 0), 0, 0), () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
Phaser globalPhaser = new Phaser();
globalPhaser.bulkRegister(3); // 2 taskExecutor threads + test thread
@@ -229,7 +231,7 @@ public void testLevelMultipliers()
throws Exception
{
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(6, 3, 3, 4, new MultilevelSplitQueue(2), ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(6, 3, 3, 4, new MultilevelSplitQueue(2), ticker);
taskExecutor.start();
try {
@@ -307,12 +309,12 @@ public void testLevelMultipliers()
public void testTaskHandle()
{
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(4, 8, 3, 4, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 8, 3, 4, ticker);
taskExecutor.start();
try {
TaskId taskId = new TaskId(new StageId("test", 0), 0, 0);
- TaskHandle taskHandle = taskExecutor.addTask(taskId, () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
+ TimeSharingTaskHandle taskHandle = taskExecutor.addTask(taskId, () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
Phaser beginPhase = new Phaser();
beginPhase.register();
@@ -343,8 +345,8 @@ public void testTaskHandle()
public void testLevelContributionCap()
{
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
- TaskHandle handle0 = new TaskHandle(new TaskId(new StageId("test0", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
- TaskHandle handle1 = new TaskHandle(new TaskId(new StageId("test1", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
+ TimeSharingTaskHandle handle0 = new TimeSharingTaskHandle(new TaskId(new StageId("test0", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
+ TimeSharingTaskHandle handle1 = new TimeSharingTaskHandle(new TaskId(new StageId("test1", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
for (int i = 0; i < (LEVEL_THRESHOLD_SECONDS.length - 1); i++) {
long levelAdvanceTime = SECONDS.toNanos(LEVEL_THRESHOLD_SECONDS[i + 1] - LEVEL_THRESHOLD_SECONDS[i]);
@@ -363,7 +365,7 @@ public void testLevelContributionCap()
public void testUpdateLevelWithCap()
{
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
- TaskHandle handle0 = new TaskHandle(new TaskId(new StageId("test0", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
+ TimeSharingTaskHandle handle0 = new TimeSharingTaskHandle(new TaskId(new StageId("test0", 0), 0, 0), splitQueue, () -> 1, 1, new Duration(1, SECONDS), OptionalInt.empty());
long quantaNanos = MINUTES.toNanos(10);
handle0.addScheduledNanos(quantaNanos);
@@ -382,7 +384,7 @@ public void testMinMaxDriversPerTask()
int maxDriversPerTask = 2;
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(4, 16, 1, maxDriversPerTask, splitQueue, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 16, 1, maxDriversPerTask, splitQueue, ticker);
taskExecutor.start();
try {
@@ -423,7 +425,7 @@ public void testUserSpecifiedMaxDriversPerTask()
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
TestingTicker ticker = new TestingTicker();
// create a task executor with min/max drivers per task to be 2 and 4
- TaskExecutor taskExecutor = new TaskExecutor(4, 16, 2, 4, splitQueue, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 16, 2, 4, splitQueue, ticker);
taskExecutor.start();
try {
@@ -463,7 +465,7 @@ public void testMinDriversPerTaskWhenTargetConcurrencyIncreases()
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
TestingTicker ticker = new TestingTicker();
// create a task executor with min/max drivers per task to be 2
- TaskExecutor taskExecutor = new TaskExecutor(4, 1, 2, 2, splitQueue, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 1, 2, 2, splitQueue, ticker);
taskExecutor.start();
try {
@@ -508,7 +510,7 @@ public void testLeafSplitsSize()
{
MultilevelSplitQueue splitQueue = new MultilevelSplitQueue(2);
TestingTicker ticker = new TestingTicker();
- TaskExecutor taskExecutor = new TaskExecutor(4, 1, 2, 2, splitQueue, ticker);
+ TimeSharingTaskExecutor taskExecutor = new TimeSharingTaskExecutor(4, 1, 2, 2, splitQueue, ticker);
TaskHandle testTaskHandle = taskExecutor.addTask(new TaskId(new StageId("test", 0), 0, 0), () -> 0, 10, new Duration(1, MILLISECONDS), OptionalInt.empty());
TestingJob driver1 = new TestingJob(ticker, new Phaser(), new Phaser(), new Phaser(), 1, 500);
diff --git a/core/trino-main/src/test/java/io/trino/execution/executor/TaskExecutorSimulator.java b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutorSimulation.java
similarity index 90%
rename from core/trino-main/src/test/java/io/trino/execution/executor/TaskExecutorSimulator.java
rename to core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutorSimulation.java
index 115fa3588dad..91942d567e62 100644
--- a/core/trino-main/src/test/java/io/trino/execution/executor/TaskExecutorSimulator.java
+++ b/core/trino-main/src/test/java/io/trino/execution/executor/timesharing/TimeSharingTaskExecutorSimulation.java
@@ -11,7 +11,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package io.trino.execution.executor;
+package io.trino.execution.executor.timesharing;
import com.google.common.base.Ticker;
import com.google.common.collect.ImmutableList;
@@ -19,14 +19,14 @@
import com.google.common.collect.ListMultimap;
import com.google.common.util.concurrent.ListeningExecutorService;
import io.airlift.units.Duration;
-import io.trino.execution.executor.SimulationController.TaskSpecification;
-import io.trino.execution.executor.SplitGenerators.AggregatedLeafSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.FastLeafSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.IntermediateSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.L4LeafSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.QuantaExceedingSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.SimpleLeafSplitGenerator;
-import io.trino.execution.executor.SplitGenerators.SlowLeafSplitGenerator;
+import io.trino.execution.executor.timesharing.SimulationController.TaskSpecification;
+import io.trino.execution.executor.timesharing.SplitGenerators.AggregatedLeafSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.FastLeafSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.IntermediateSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.L4LeafSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.QuantaExceedingSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.SimpleLeafSplitGenerator;
+import io.trino.execution.executor.timesharing.SplitGenerators.SlowLeafSplitGenerator;
import org.joda.time.DateTime;
import java.io.Closeable;
@@ -42,10 +42,10 @@
import static io.airlift.concurrent.Threads.threadsNamed;
import static io.airlift.units.Duration.nanosSince;
import static io.airlift.units.Duration.succinctNanos;
-import static io.trino.execution.executor.Histogram.fromContinuous;
-import static io.trino.execution.executor.Histogram.fromDiscrete;
-import static io.trino.execution.executor.SimulationController.TaskSpecification.Type.INTERMEDIATE;
-import static io.trino.execution.executor.SimulationController.TaskSpecification.Type.LEAF;
+import static io.trino.execution.executor.timesharing.Histogram.fromContinuous;
+import static io.trino.execution.executor.timesharing.Histogram.fromDiscrete;
+import static io.trino.execution.executor.timesharing.SimulationController.TaskSpecification.Type.INTERMEDIATE;
+import static io.trino.execution.executor.timesharing.SimulationController.TaskSpecification.Type.LEAF;
import static java.lang.String.format;
import static java.util.concurrent.Executors.newCachedThreadPool;
import static java.util.concurrent.Executors.newScheduledThreadPool;
@@ -57,13 +57,13 @@
import static java.util.concurrent.TimeUnit.SECONDS;
import static java.util.function.Function.identity;
-public class TaskExecutorSimulator
+public class TimeSharingTaskExecutorSimulation
implements Closeable
{
public static void main(String[] args)
throws Exception
{
- try (TaskExecutorSimulator simulator = new TaskExecutorSimulator()) {
+ try (TimeSharingTaskExecutorSimulation simulator = new TimeSharingTaskExecutorSimulation()) {
simulator.run();
}
}
@@ -73,13 +73,13 @@ public static void main(String[] args)
private final ScheduledExecutorService runningSplitsPrintExecutor = newSingleThreadScheduledExecutor();
private final ScheduledExecutorService wakeupExecutor = newScheduledThreadPool(32);
- private final TaskExecutor taskExecutor;
+ private final TimeSharingTaskExecutor taskExecutor;
private final MultilevelSplitQueue splitQueue;
- private TaskExecutorSimulator()
+ private TimeSharingTaskExecutorSimulation()
{
splitQueue = new MultilevelSplitQueue(2);
- taskExecutor = new TaskExecutor(36, 72, 3, 8, splitQueue, Ticker.systemTicker());
+ taskExecutor = new TimeSharingTaskExecutor(36, 72, 3, 8, splitQueue, Ticker.systemTicker());
taskExecutor.start();
}
@@ -99,7 +99,7 @@ public void run()
long start = System.nanoTime();
scheduleStatusPrinter(start);
- SimulationController controller = new SimulationController(taskExecutor, TaskExecutorSimulator::printSummaryStats);
+ SimulationController controller = new SimulationController(taskExecutor, TimeSharingTaskExecutorSimulation::printSummaryStats);
// Uncomment one of these:
// runExperimentOverloadedCluster(controller);
@@ -306,7 +306,7 @@ private void scheduleStatusPrinter(long start)
}, 1, 1, SECONDS);
}
- private static void printSummaryStats(SimulationController controller, TaskExecutor taskExecutor)
+ private static void printSummaryStats(SimulationController controller, TimeSharingTaskExecutor taskExecutor)
{
Map specEnabled = controller.getSpecificationEnabled();
@@ -350,7 +350,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getProcessedTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Levels - Running Task Processed Time");
@@ -359,7 +359,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getProcessedTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Levels - All Task Wait Time");
@@ -368,7 +368,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getTotalWaitTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Specification - Processed time");
@@ -378,7 +378,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
t -> t.getSpecification().getName(),
SimulationTask::getProcessedTimeNanos,
identity(),
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Specification - Wait time");
@@ -387,7 +387,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
t -> t.getSpecification().getName(),
SimulationTask::getTotalWaitTimeNanos,
identity(),
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Breakdown by specification");
@@ -415,7 +415,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getProcessedTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("All Tasks by Scheduled time - Wait Time");
@@ -424,7 +424,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getTotalWaitTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
System.out.println();
System.out.println("Complete Tasks by Scheduled time - Wait Time");
@@ -433,7 +433,7 @@ private static void printSummaryStats(SimulationController controller, TaskExecu
SimulationTask::getScheduledTimeNanos,
SimulationTask::getTotalWaitTimeNanos,
Duration::succinctNanos,
- TaskExecutorSimulator::formatNanos);
+ TimeSharingTaskExecutorSimulation::formatNanos);
}
}
diff --git a/core/trino-main/src/test/java/io/trino/operator/project/TestPageProcessor.java b/core/trino-main/src/test/java/io/trino/operator/project/TestPageProcessor.java
index 512f7df33832..e8cae580be26 100644
--- a/core/trino-main/src/test/java/io/trino/operator/project/TestPageProcessor.java
+++ b/core/trino-main/src/test/java/io/trino/operator/project/TestPageProcessor.java
@@ -51,7 +51,7 @@
import static io.trino.block.BlockAssertions.createLongSequenceBlock;
import static io.trino.block.BlockAssertions.createSlicesBlock;
import static io.trino.block.BlockAssertions.createStringsBlock;
-import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
+import static io.trino.execution.executor.timesharing.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext;
import static io.trino.operator.PageAssertions.assertPageEquals;
import static io.trino.operator.project.PageProcessor.MAX_BATCH_SIZE;
diff --git a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java
index f15051a0b84d..d5082fa78a39 100644
--- a/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java
+++ b/core/trino-main/src/test/java/io/trino/server/remotetask/TestHttpRemoteTask.java
@@ -35,6 +35,7 @@
import io.trino.Session;
import io.trino.block.BlockJsonSerde;
import io.trino.client.NodeVersion;
+import io.trino.execution.BaseTestSqlTaskManager;
import io.trino.execution.DynamicFilterConfig;
import io.trino.execution.DynamicFiltersCollector.VersionedDynamicFilterDomains;
import io.trino.execution.NodeTaskMap;
@@ -49,7 +50,6 @@
import io.trino.execution.TaskState;
import io.trino.execution.TaskStatus;
import io.trino.execution.TaskTestUtils;
-import io.trino.execution.TestSqlTaskManager;
import io.trino.execution.buffer.PipelinedOutputBuffers;
import io.trino.metadata.BlockEncodingManager;
import io.trino.metadata.HandleJsonModule;
@@ -583,7 +583,7 @@ private HttpRemoteTaskFactory createHttpRemoteTaskFactory(
new QueryManagerConfig(),
TASK_MANAGER_CONFIG,
testingHttpClient,
- new TestSqlTaskManager.MockLocationFactory(),
+ new BaseTestSqlTaskManager.MockLocationFactory(),
taskStatusCodec,
dynamicFilterDomainsCodec,
taskInfoCodec,
diff --git a/pom.xml b/pom.xml
index 42d224497844..1b97f9a074ac 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2497,7 +2497,8 @@
-Xep:StreamResourceLeak:ERROR \
-Xep:UnnecessaryMethodReference:ERROR \
-Xep:UnnecessaryOptionalGet:ERROR \
- -Xep:UnusedVariable:ERROR \
+
+
-Xep:UseEnumSwitch:ERROR \
-XepExcludedPaths:.*/target/generated-(|test-)sources/.*
diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractOperatorBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractOperatorBenchmark.java
index 186d3b0edb20..558d217c0d14 100644
--- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractOperatorBenchmark.java
+++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractOperatorBenchmark.java
@@ -81,7 +81,7 @@
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageRowCount;
import static io.trino.SystemSessionProperties.getFilterAndProjectMinOutputPageSize;
-import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
+import static io.trino.execution.executor.timesharing.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
import static io.trino.spi.connector.Constraint.alwaysTrue;
import static io.trino.spi.type.BigintType.BIGINT;
import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes;
diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java
index 87216d0f284d..900e362b6201 100644
--- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java
+++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashJoinBenchmark.java
@@ -41,7 +41,7 @@
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.airlift.concurrent.MoreFutures.getFutureValue;
import static io.trino.benchmark.BenchmarkQueryRunner.createLocalQueryRunner;
-import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
+import static io.trino.execution.executor.timesharing.PrioritizedSplitRunner.SPLIT_RUN_QUANTA;
import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier;
import static io.trino.operator.JoinOperatorType.innerJoin;
import static io.trino.operator.OperatorFactories.spillingJoin;