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
30 changes: 30 additions & 0 deletions core/trino-main/src/main/java/io/trino/FeaturesConfig.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@
import java.util.List;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.airlift.units.DataSize.Unit.GIGABYTE;
import static io.airlift.units.DataSize.Unit.KILOBYTE;
import static io.airlift.units.DataSize.Unit.MEGABYTE;
import static io.trino.sql.analyzer.RegexLibrary.JONI;
Expand Down Expand Up @@ -152,6 +153,9 @@ public class FeaturesConfig
private Duration retryInitialDelay = new Duration(10, SECONDS);
private Duration retryMaxDelay = new Duration(1, MINUTES);

private DataSize faultTolerantExecutionTargetTaskInputSize = DataSize.of(1, GIGABYTE);
private int faultTolerantExecutionTargetTaskSplitCount = 16;

public enum JoinReorderingStrategy
{
NONE,
Expand Down Expand Up @@ -1169,4 +1173,30 @@ public FeaturesConfig setRetryMaxDelay(Duration retryMaxDelay)
this.retryMaxDelay = retryMaxDelay;
return this;
}

@NotNull
public DataSize getFaultTolerantExecutionTargetTaskInputSize()
{
return faultTolerantExecutionTargetTaskInputSize;
}

@Config("fault-tolerant-execution-target-task-input-size")
public FeaturesConfig setFaultTolerantExecutionTargetTaskInputSize(DataSize faultTolerantExecutionTargetTaskInputSize)
{
this.faultTolerantExecutionTargetTaskInputSize = faultTolerantExecutionTargetTaskInputSize;
return this;
}

@Min(1)
public int getFaultTolerantExecutionTargetTaskSplitCount()
{
return faultTolerantExecutionTargetTaskSplitCount;
}

@Config("fault-tolerant-execution-target-task-split-count")
public FeaturesConfig setFaultTolerantExecutionTargetTaskSplitCount(int faultTolerantExecutionTargetTaskSplitCount)
{
this.faultTolerantExecutionTargetTaskSplitCount = faultTolerantExecutionTargetTaskSplitCount;
return this;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,8 @@ public final class SystemSessionProperties
public static final String RETRY_ATTEMPTS = "retry_attempts";
public static final String RETRY_INITIAL_DELAY = "retry_initial_delay";
public static final String RETRY_MAX_DELAY = "retry_max_delay";
public static final String FAULT_TOLERANT_EXECUTION_TARGET_TASK_INPUT_SIZE = "fault_tolerant_execution_target_task_input_size";
public static final String FAULT_TOLERANT_EXECUTION_TARGET_TASK_SPLIT_COUNT = "fault_tolerant_execution_target_task_split_count";

private final List<PropertyMetadata<?>> sessionProperties;

Expand Down Expand Up @@ -692,6 +694,16 @@ public SystemSessionProperties(
RETRY_MAX_DELAY,
"Maximum delay before initiating a retry attempt. Delay increases exponentially for each subsequent attempt starting from 'retry_initial_delay'",
featuresConfig.getRetryMaxDelay(),
false),
dataSizeProperty(
FAULT_TOLERANT_EXECUTION_TARGET_TASK_INPUT_SIZE,
"Target size of all task inputs for a single fault tolerant task",
featuresConfig.getFaultTolerantExecutionTargetTaskInputSize(),
false),
integerProperty(
FAULT_TOLERANT_EXECUTION_TARGET_TASK_SPLIT_COUNT,
"Target number of splits for a single fault tolerant task",
featuresConfig.getFaultTolerantExecutionTargetTaskSplitCount(),
false));
}

Expand Down Expand Up @@ -1222,6 +1234,11 @@ public static RetryPolicy getRetryPolicy(Session session)
throw new TrinoException(NOT_SUPPORTED, "Distributed sort is not supported with automatic retries enabled");
}
}
if (retryPolicy == RetryPolicy.TASK) {
if (isGroupedExecutionEnabled(session) || isDynamicScheduleForGroupedExecution(session)) {
throw new TrinoException(NOT_SUPPORTED, "Grouped execution is not supported with task level retries enabled");
}
}
return retryPolicy;
}

Expand All @@ -1239,4 +1256,14 @@ public static Duration getRetryMaxDelay(Session session)
{
return session.getSystemProperty(RETRY_MAX_DELAY, Duration.class);
}

public static DataSize getFaultTolerantExecutionTargetTaskInputSize(Session session)
{
return session.getSystemProperty(FAULT_TOLERANT_EXECUTION_TARGET_TASK_INPUT_SIZE, DataSize.class);
}

public static int getFaultTolerantExecutionTargetTaskSplitCount(Session session)
{
return session.getSystemProperty(FAULT_TOLERANT_EXECUTION_TARGET_TASK_SPLIT_COUNT, Integer.class);
}
}
Original file line number Diff line number Diff line change
@@ -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.exchange;

import com.google.inject.Binder;
import com.google.inject.Module;
import com.google.inject.Scopes;

public class ExchangeManagerModule
implements Module
{
@Override
public void configure(Binder binder)
{
binder.bind(ExchangeManagerRegistry.class).in(Scopes.SINGLETON);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
/*
* 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.exchange;

import io.airlift.log.Logger;
import io.trino.metadata.HandleResolver;
import io.trino.spi.TrinoException;
import io.trino.spi.classloader.ThreadContextClassLoader;
import io.trino.spi.exchange.ExchangeManager;
import io.trino.spi.exchange.ExchangeManagerFactory;

import javax.annotation.concurrent.GuardedBy;
import javax.inject.Inject;

import java.io.File;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Preconditions.checkState;
import static com.google.common.base.Strings.isNullOrEmpty;
import static io.airlift.configuration.ConfigurationLoader.loadPropertiesFrom;
import static io.trino.spi.StandardErrorCode.EXCHANGE_MANAGER_NOT_CONFIGURED;
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;

public class ExchangeManagerRegistry
{
private static final Logger log = Logger.get(ExchangeManagerRegistry.class);

private static final File CONFIG_FILE = new File("etc/exchange-manager.properties");
private static final String EXCHANGE_MANAGER_NAME_PROPERTY = "exchange-manager.name";

private final HandleResolver handleResolver;

private final Map<String, ExchangeManagerFactory> exchangeManagerFactories = new ConcurrentHashMap<>();

@GuardedBy("this")
private volatile ExchangeManager exchangeManager;

@Inject
public ExchangeManagerRegistry(HandleResolver handleResolver)
{
this.handleResolver = requireNonNull(handleResolver, "handleResolver is null");
}

public void addExchangeManagerFactory(ExchangeManagerFactory factory)
{
requireNonNull(factory, "factory is null");
if (exchangeManagerFactories.putIfAbsent(factory.getName(), factory) != null) {
throw new IllegalArgumentException(format("Exchange manager factory '%s' is already registered", factory.getName()));
}
}

public void loadExchangeManager()
{
if (!CONFIG_FILE.exists()) {
log.info("Exchange manager configuration file is not present: %s", CONFIG_FILE.getAbsoluteFile());
return;
}

Map<String, String> properties = loadProperties(CONFIG_FILE);
String name = properties.remove(EXCHANGE_MANAGER_NAME_PROPERTY);
checkArgument(!isNullOrEmpty(name), "Exchange manager configuration %s does not contain %s", CONFIG_FILE, EXCHANGE_MANAGER_NAME_PROPERTY);

loadExchangeManager(name, properties);
}

public synchronized void loadExchangeManager(String name, Map<String, String> properties)
{
log.info("-- Loading exchange manager %s --", name);

checkState(exchangeManager == null, "exchangeManager is already loaded");

ExchangeManagerFactory factory = exchangeManagerFactories.get(name);
checkArgument(factory != null, "Exchange manager factory '%s' is not registered. Available factories: %s", name, exchangeManagerFactories.keySet());

ExchangeManager exchangeManager;
try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(factory.getClass().getClassLoader())) {
exchangeManager = factory.create(properties);
}
handleResolver.setExchangeManagerHandleResolver(factory.getHandleResolver());

log.info("-- Loaded exchange manager %s --", name);

this.exchangeManager = exchangeManager;
}

public ExchangeManager getExchangeManager()
{
ExchangeManager exchangeManager = this.exchangeManager;
if (exchangeManager == null) {
throw new TrinoException(EXCHANGE_MANAGER_NOT_CONFIGURED, "Exchange manager is not configured");
}
return exchangeManager;
}

private static Map<String, String> loadProperties(File configFile)
{
try {
return new HashMap<>(loadPropertiesFrom(configFile.getPath()));
}
catch (IOException e) {
throw new UncheckedIOException("Failed to read configuration file: " + configFile, e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import com.google.common.collect.ImmutableList;
import io.trino.client.ErrorLocation;
import io.trino.client.FailureInfo;
import io.trino.failuredetector.FailureDetector;
import io.trino.spi.ErrorCode;
import io.trino.spi.HostAddress;

Expand All @@ -29,6 +30,8 @@
import java.util.regex.Pattern;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.failuredetector.FailureDetector.State.GONE;
import static io.trino.spi.StandardErrorCode.REMOTE_HOST_GONE;
import static java.util.Objects.requireNonNull;

@Immutable
Expand Down Expand Up @@ -175,4 +178,21 @@ else if (matcher.group(4) != null) {
}
return new StackTraceElement("Unknown", stack, null, -1);
}

public static ExecutionFailureInfo rewriteTransportFailure(FailureDetector failureDetector, ExecutionFailureInfo executionFailureInfo)
{
if (executionFailureInfo.getRemoteHost() == null || failureDetector.getState(executionFailureInfo.getRemoteHost()) != GONE) {
return executionFailureInfo;
}

return new ExecutionFailureInfo(
executionFailureInfo.getType(),
executionFailureInfo.getMessage(),
executionFailureInfo.getCause(),
executionFailureInfo.getSuppressed(),
executionFailureInfo.getStack(),
executionFailureInfo.getErrorLocation(),
REMOTE_HOST_GONE.toErrorCode(),
executionFailureInfo.getRemoteHost());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -1184,7 +1184,7 @@ private static QueryStats pruneQueryStats(QueryStats queryStats)
queryStats.getPhysicalWrittenDataSize(),
queryStats.getStageGcStatistics(),
queryStats.getDynamicFiltersStats(),
ImmutableList.of()); // Remove the operator summaries as OperatorInfo (especially ExchangeClientStatus) can hold onto a large amount of memory
ImmutableList.of()); // Remove the operator summaries as OperatorInfo (especially DirectExchangeClientStatus) can hold onto a large amount of memory
}

public static class QueryOutputManager
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,15 @@
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Objects.requireNonNull;

public class TaskSource
public class SplitAssignment
{
private final PlanNodeId planNodeId;
private final Set<ScheduledSplit> splits;
private final Set<Lifespan> noMoreSplitsForLifespan;
private final boolean noMoreSplits;

@JsonCreator
public TaskSource(
public SplitAssignment(
@JsonProperty("planNodeId") PlanNodeId planNodeId,
@JsonProperty("splits") Set<ScheduledSplit> splits,
@JsonProperty("noMoreSplitsForLifespan") Set<Lifespan> noMoreSplitsForLifespan,
Expand All @@ -44,7 +44,7 @@ public TaskSource(
this.noMoreSplits = noMoreSplits;
}

public TaskSource(PlanNodeId planNodeId, Set<ScheduledSplit> splits, boolean noMoreSplits)
public SplitAssignment(PlanNodeId planNodeId, Set<ScheduledSplit> splits, boolean noMoreSplits)
{
this(planNodeId, splits, ImmutableSet.of(), noMoreSplits);
}
Expand Down Expand Up @@ -73,43 +73,43 @@ public boolean isNoMoreSplits()
return noMoreSplits;
}

public TaskSource update(TaskSource source)
public SplitAssignment update(SplitAssignment assignment)
{
checkArgument(planNodeId.equals(source.getPlanNodeId()), "Expected source %s, but got source %s", planNodeId, source.getPlanNodeId());
checkArgument(planNodeId.equals(assignment.getPlanNodeId()), "Expected assignment for node %s, but got assignment for node %s", planNodeId, assignment.getPlanNodeId());

if (isNewer(source)) {
// assure the new source is properly formed
// we know that either the new source one has new splits and/or it is marking the source as closed
checkArgument(!noMoreSplits || splits.containsAll(source.getSplits()), "Source %s has new splits, but no more splits already set", planNodeId);
if (isNewer(assignment)) {
// assure the new assignment is properly formed
// we know that either the new assignment one has new splits and/or it is marking the assignment as closed
checkArgument(!noMoreSplits || splits.containsAll(assignment.getSplits()), "Assignment %s has new splits, but no more splits already set", planNodeId);

Set<ScheduledSplit> newSplits = ImmutableSet.<ScheduledSplit>builder()
.addAll(splits)
.addAll(source.getSplits())
.addAll(assignment.getSplits())
.build();
Set<Lifespan> newNoMoreSplitsForDriverGroup = ImmutableSet.<Lifespan>builder()
.addAll(noMoreSplitsForLifespan)
.addAll(source.getNoMoreSplitsForLifespan())
.addAll(assignment.getNoMoreSplitsForLifespan())
.build();

return new TaskSource(
return new SplitAssignment(
planNodeId,
newSplits,
newNoMoreSplitsForDriverGroup,
source.isNoMoreSplits());
assignment.isNoMoreSplits());
}
else {
// the specified source is older than this one
// the specified assignment is older than this one
return this;
}
}

private boolean isNewer(TaskSource source)
private boolean isNewer(SplitAssignment assignment)
{
// the specified source is newer if it changes the no more
// the specified assignment is newer if it changes the no more
// splits flag or if it contains new splits
return (!noMoreSplits && source.isNoMoreSplits()) ||
(!noMoreSplitsForLifespan.containsAll(source.getNoMoreSplitsForLifespan())) ||
(!splits.containsAll(source.getSplits()));
return (!noMoreSplits && assignment.isNoMoreSplits()) ||
(!noMoreSplitsForLifespan.containsAll(assignment.getNoMoreSplitsForLifespan())) ||
(!splits.containsAll(assignment.getSplits()));
}

@Override
Expand Down
Loading