Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ public final class SystemSessionProperties
public static final String STATISTICS_PRECALCULATION_FOR_PUSHDOWN_ENABLED = "statistics_precalculation_for_pushdown_enabled";
public static final String COLLECT_PLAN_STATISTICS_FOR_ALL_QUERIES = "collect_plan_statistics_for_all_queries";
public static final String IGNORE_STATS_CALCULATOR_FAILURES = "ignore_stats_calculator_failures";
public static final String MAX_DRIVERS_PER_QUERY = "max_drivers_per_query";
public static final String MAX_DRIVERS_PER_TASK = "max_drivers_per_task";
public static final String DEFAULT_FILTER_FACTOR_ENABLED = "default_filter_factor_enabled";
public static final String SKIP_REDUNDANT_SORT = "skip_redundant_sort";
Expand Down Expand Up @@ -519,6 +520,11 @@ public SystemSessionProperties(
"Collect plan statistics for non-EXPLAIN queries",
featuresConfig.isCollectPlanStatisticsForAllQueries(),
false),
integerProperty(
MAX_DRIVERS_PER_QUERY,
"Maximum number of drivers per query",
taskManagerConfig.getMaxDriversPerQuery(),
false),
new PropertyMetadata<>(
MAX_DRIVERS_PER_TASK,
"Maximum number of drivers per task",
Expand Down Expand Up @@ -1034,6 +1040,11 @@ private static void validateHideInaccesibleColumns(boolean value, boolean defaul
}
}

public static int getMaxDriversPerQuery(Session session)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You should only be able to reduce max_drivers_per_query and not increase it vs value in FeatureConfig

{
return session.getSystemProperty(MAX_DRIVERS_PER_QUERY, Integer.class);
}

public static OptionalInt getMaxDriversPerTask(Session session)
{
Integer value = session.getSystemProperty(MAX_DRIVERS_PER_TASK, Integer.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,7 @@
import static com.google.common.collect.ImmutableMap.toImmutableMap;
import static com.google.common.util.concurrent.Futures.immediateVoidFuture;
import static io.trino.SystemSessionProperties.getInitialSplitsPerNode;
import static io.trino.SystemSessionProperties.getMaxDriversPerQuery;
import static io.trino.SystemSessionProperties.getMaxDriversPerTask;
import static io.trino.SystemSessionProperties.getSplitConcurrencyAdjustmentInterval;
import static io.trino.execution.SqlTaskExecution.SplitsState.ADDING_SPLITS;
Expand Down Expand Up @@ -263,7 +264,8 @@ private static TaskHandle createTaskHandle(
outputBuffer::getUtilization,
getInitialSplitsPerNode(taskContext.getSession()),
getSplitConcurrencyAdjustmentInterval(taskContext.getSession()),
getMaxDriversPerTask(taskContext.getSession()));
getMaxDriversPerTask(taskContext.getSession()),
getMaxDriversPerQuery(taskContext.getSession()));
taskStateMachine.addStateChangeListener(state -> {
if (state.isDone()) {
taskExecutor.removeTask(taskHandle);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,7 @@ public class TaskManagerConfig
private boolean shareIndexLoading;
private int maxWorkerThreads = Runtime.getRuntime().availableProcessors() * 2;
private Integer minDrivers;
private Integer maxDriversPerQuery;
private Integer initialSplitsPerNode;
private int minDriversPerTask = 3;
private int maxDriversPerTask = Integer.MAX_VALUE;
Expand Down Expand Up @@ -287,6 +288,24 @@ public TaskManagerConfig setMinDrivers(int minDrivers)
return this;
}

@Min(1)
public int getMaxDriversPerQuery()
{
if (maxDriversPerQuery == null) {
// minDrivers has the higher priority over maxDriversPerQuery.
// That means maxDriversPerQuery is capped by minDrivers.
return getMinDrivers();
}
return maxDriversPerQuery;
}

@Config("task.max-drivers-per-query")
public TaskManagerConfig setMaxDriversPerQuery(int maxDrivers)
{
this.maxDriversPerQuery = maxDrivers;
return this;
}

@Min(1)
public int getMaxDriversPerTask()
{
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,85 @@
/*
* 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 java.util.concurrent.atomic.AtomicInteger;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;

public class DriverLimitPerQuery
{
private static final String ACCESS_TO_UNREFERENCED_INSTANCE_ERROR_MESSAGE = "Attempt to access unreferenced instance";

private final int maxDriversPerQuery;
private final AtomicInteger numberOfDrivers;
private final AtomicInteger referenceCount;

public DriverLimitPerQuery(int maxDriversPerQuery)
{
this.maxDriversPerQuery = maxDriversPerQuery;
numberOfDrivers = new AtomicInteger(0);
referenceCount = new AtomicInteger(0);
}

public void increase()
{
checkSanity();
checkState(numberOfDrivers.getAndIncrement() >= 0, "numberOfDrivers is a negative number");
}

public void decrease()
{
checkSanity();
checkState(numberOfDrivers.decrementAndGet() >= 0, "numberOfDrivers turned into a negative number");
}

public void subtract(int delta)
{
checkSanity();
checkArgument(delta > 0, "delta is equal to or less than zero");
checkState(numberOfDrivers.addAndGet(delta * -1) >= 0, "numberOfDrivers turned into a negative number");
}

public boolean isFull()
{
checkSanity();
return numberOfDrivers.get() >= maxDriversPerQuery;
}

public void addInitialReference()
{
checkState(referenceCount.getAndIncrement() == 0, "referenceCount is non-zero when initialize");
}

public void addReference()
{
checkState(referenceCount.getAndIncrement() > 0, ACCESS_TO_UNREFERENCED_INSTANCE_ERROR_MESSAGE);
}

public boolean dereference()
{
int currentCount = referenceCount.decrementAndGet();
if (currentCount < 0) {
throw new IllegalStateException(ACCESS_TO_UNREFERENCED_INSTANCE_ERROR_MESSAGE);
}
// is unreferenced?
return currentCount == 0;
}

private void checkSanity()
{
checkState(referenceCount.get() > 0, ACCESS_TO_UNREFERENCED_INSTANCE_ERROR_MESSAGE);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import io.trino.execution.SplitRunner;
import io.trino.execution.TaskId;
import io.trino.execution.TaskManagerConfig;
import io.trino.spi.QueryId;
import io.trino.spi.TrinoException;
import io.trino.spi.VersionEmbedder;
import org.weakref.jmx.Managed;
Expand All @@ -39,6 +40,7 @@
import javax.inject.Inject;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedList;
Expand Down Expand Up @@ -154,6 +156,9 @@ public class TaskExecutor
private final TimeStat blockedQuantaWallTime = new TimeStat(MICROSECONDS);
private final TimeStat unblockedQuantaWallTime = new TimeStat(MICROSECONDS);

@GuardedBy("this")
private final Map<QueryId, DriverLimitPerQuery> queryIdDriverLimitPerQueryMap = new HashMap<>();

private volatile boolean closed;

@Inject
Expand Down Expand Up @@ -254,7 +259,8 @@ public synchronized TaskHandle addTask(
DoubleSupplier utilizationSupplier,
int initialSplitConcurrency,
Duration splitConcurrencyAdjustFrequency,
OptionalInt maxDriversPerTask)
OptionalInt maxDriversPerTask,
int maxDriversPerQuery)
{
requireNonNull(taskId, "taskId is null");
requireNonNull(utilizationSupplier, "utilizationSupplier is null");
Expand All @@ -263,7 +269,24 @@ public synchronized TaskHandle addTask(

log.debug("Task scheduled %s", taskId);

TaskHandle taskHandle = new TaskHandle(taskId, waitingSplits, utilizationSupplier, initialSplitConcurrency, splitConcurrencyAdjustFrequency, maxDriversPerTask);
QueryId queryId = taskId.getQueryId();
DriverLimitPerQuery driverLimitPerQuery = queryIdDriverLimitPerQueryMap.get(queryId);
if (driverLimitPerQuery == null) {
driverLimitPerQuery = new DriverLimitPerQuery(maxDriversPerQuery);
driverLimitPerQuery.addInitialReference();
queryIdDriverLimitPerQueryMap.put(queryId, driverLimitPerQuery);
}
else {
driverLimitPerQuery.addReference();
}
TaskHandle taskHandle = new TaskHandle(
taskId,
waitingSplits,
utilizationSupplier,
initialSplitConcurrency,
splitConcurrencyAdjustFrequency,
maxDriversPerTask,
driverLimitPerQuery);

tasks.add(taskHandle);
return taskHandle;
Expand All @@ -285,6 +308,9 @@ private void doRemoveTask(TaskHandle taskHandle)
synchronized (this) {
tasks.remove(taskHandle);
splits = taskHandle.destroy();
if (taskHandle.dereferenceFromDriverLimitPerQuery()) {
queryIdDriverLimitPerQueryMap.remove(taskHandle.getTaskId().getQueryId());
}

// stop tracking splits (especially blocked splits which may never unblock)
allSplits.removeAll(splits);
Expand Down Expand Up @@ -443,6 +469,10 @@ private synchronized PrioritizedSplitRunner pollNextSplitWorker()
if (task.getRunningLeafSplits() >= task.getMaxDriversPerTask().orElse(maximumNumberOfDriversPerTask)) {
continue;
}
// skip tasks whose max number of drivers per query value is equal to or less than the current total running leaf splits
if (task.isDriverLimitPerQueryExceeded()) {
continue;
}
PrioritizedSplitRunner split = task.pollNextSplit();
if (split != null) {
// move task to end of list
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,19 +58,22 @@ public class TaskHandle
protected final AtomicReference<Priority> priority = new AtomicReference<>(new Priority(0, 0));
private final MultilevelSplitQueue splitQueue;
private final OptionalInt maxDriversPerTask;
private final DriverLimitPerQuery driverLimitPerQuery;

public TaskHandle(
TaskId taskId,
MultilevelSplitQueue splitQueue,
DoubleSupplier utilizationSupplier,
int initialSplitConcurrency,
Duration splitConcurrencyAdjustFrequency,
OptionalInt maxDriversPerTask)
OptionalInt maxDriversPerTask,
DriverLimitPerQuery driverLimitPerQuery)
{
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.driverLimitPerQuery = requireNonNull(driverLimitPerQuery, "driverLimitPerQuery is null");
this.concurrencyController = new SplitConcurrencyController(
initialSplitConcurrency,
requireNonNull(splitConcurrencyAdjustFrequency, "splitConcurrencyAdjustFrequency is null"));
Expand Down Expand Up @@ -114,6 +117,16 @@ public TaskId getTaskId()
return taskId;
}

public boolean isDriverLimitPerQueryExceeded()
{
return driverLimitPerQuery.isFull();
}

public boolean dereferenceFromDriverLimitPerQuery()
{
return driverLimitPerQuery.dereference();
}

public OptionalInt getMaxDriversPerTask()
{
return maxDriversPerTask;
Expand All @@ -123,6 +136,10 @@ public OptionalInt getMaxDriversPerTask()
public synchronized List<PrioritizedSplitRunner> destroy()
{
destroyed = true;
int runningLeafSplitsSize = runningLeafSplits.size();
if (runningLeafSplitsSize > 0) {
driverLimitPerQuery.subtract(runningLeafSplitsSize);
}

ImmutableList.Builder<PrioritizedSplitRunner> builder = ImmutableList.builder();
builder.addAll(runningIntermediateSplits);
Expand Down Expand Up @@ -169,6 +186,7 @@ public synchronized PrioritizedSplitRunner pollNextSplit()
PrioritizedSplitRunner split = queuedLeafSplits.poll();
if (split != null) {
runningLeafSplits.add(split);
driverLimitPerQuery.increase();
}
return split;
}
Expand All @@ -177,7 +195,9 @@ public synchronized void splitComplete(PrioritizedSplitRunner split)
{
concurrencyController.splitFinished(split.getScheduledNanos(), utilizationSupplier.getAsDouble(), runningLeafSplits.size());
runningIntermediateSplits.remove(split);
runningLeafSplits.remove(split);
if (runningLeafSplits.remove(split)) {
driverLimitPerQuery.decrease();
}
}

public int getNextSplitId()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@ public void testDefaults()
.setTaskCpuTimerEnabled(true)
.setMaxWorkerThreads(Runtime.getRuntime().availableProcessors() * 2)
.setMinDrivers(Runtime.getRuntime().availableProcessors() * 2 * 2)
.setMaxDriversPerQuery(Runtime.getRuntime().availableProcessors() * 2 * 2)
.setMinDriversPerTask(3)
.setMaxDriversPerTask(Integer.MAX_VALUE)
.setInfoMaxAge(new Duration(15, TimeUnit.MINUTES))
Expand Down Expand Up @@ -86,6 +87,7 @@ public void testExplicitPropertyMappings()
.put("task.max-local-exchange-buffer-size", "33MB")
.put("task.max-worker-threads", "3")
.put("task.min-drivers", "2")
.put("task.max-drivers-per-query", "2")
.put("task.min-drivers-per-task", "5")
.put("task.max-drivers-per-task", "13")
.put("task.info.max-age", "22m")
Expand Down Expand Up @@ -117,6 +119,7 @@ public void testExplicitPropertyMappings()
.setMaxLocalExchangeBufferSize(DataSize.of(33, Unit.MEGABYTE))
.setMaxWorkerThreads(3)
.setMinDrivers(2)
.setMaxDriversPerQuery(2)
.setMinDriversPerTask(5)
.setMaxDriversPerTask(13)
.setInfoMaxAge(new Duration(22, TimeUnit.MINUTES))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public SimulationTask(TaskExecutor taskExecutor, TaskSpecification specification
{
this.specification = specification;
this.taskId = taskId;
taskHandle = taskExecutor.addTask(taskId, () -> 0, 10, new Duration(1, SECONDS), OptionalInt.empty());
taskHandle = taskExecutor.addTask(taskId, () -> 0, 10, new Duration(1, SECONDS), OptionalInt.empty(), Integer.MAX_VALUE);
}

public void setKilled()
Expand Down
Loading