Skip to content
Merged
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 @@ -91,6 +91,7 @@ public void addInput(ExchangeInput input)
if (exchangeSource == null) {
return;
}
spoolingExchangeInput.getOutputSelector().ifPresent(exchangeSource::setOutputSelector);
exchangeSource.addSourceHandles(spoolingExchangeInput.getExchangeSourceHandles());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,15 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import io.trino.spi.exchange.ExchangeSourceHandle;
import io.trino.spi.exchange.ExchangeSourceOutputSelector;
import org.openjdk.jol.info.ClassLayout;

import java.util.List;
import java.util.Optional;

import static com.google.common.base.MoreObjects.toStringHelper;
import static io.airlift.slice.SizeOf.estimatedSizeOf;
import static io.airlift.slice.SizeOf.sizeOf;
import static java.lang.Math.toIntExact;
import static java.util.Objects.requireNonNull;

Expand All @@ -32,11 +35,15 @@ public class SpoolingExchangeInput
private static final int INSTANCE_SIZE = toIntExact(ClassLayout.parseClass(SpoolingExchangeInput.class).instanceSize());

private final List<ExchangeSourceHandle> exchangeSourceHandles;
private final Optional<ExchangeSourceOutputSelector> outputSelector;

@JsonCreator
public SpoolingExchangeInput(@JsonProperty("exchangeSourceHandles") List<ExchangeSourceHandle> exchangeSourceHandles)
public SpoolingExchangeInput(
@JsonProperty("exchangeSourceHandles") List<ExchangeSourceHandle> exchangeSourceHandles,
@JsonProperty("outputSelector") Optional<ExchangeSourceOutputSelector> outputSelector)
{
this.exchangeSourceHandles = ImmutableList.copyOf(requireNonNull(exchangeSourceHandles, "exchangeSourceHandles is null"));
this.outputSelector = requireNonNull(outputSelector, "outputSelector is null");
}

@JsonProperty
Expand All @@ -45,18 +52,26 @@ public List<ExchangeSourceHandle> getExchangeSourceHandles()
return exchangeSourceHandles;
}

@JsonProperty
public Optional<ExchangeSourceOutputSelector> getOutputSelector()
{
return outputSelector;
}

@Override
public String toString()
{
return toStringHelper(this)
.add("exchangeSourceHandles", exchangeSourceHandles)
.add("outputSelector", outputSelector)
.toString();
}

@Override
public long getRetainedSizeInBytes()
{
return INSTANCE_SIZE
+ estimatedSizeOf(exchangeSourceHandles, ExchangeSourceHandle::getRetainedSizeInBytes);
+ estimatedSizeOf(exchangeSourceHandles, ExchangeSourceHandle::getRetainedSizeInBytes)
+ sizeOf(outputSelector, ExchangeSourceOutputSelector::getRetainedSizeInBytes);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,14 +15,13 @@

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.Futures;
import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.ListenableFuture;
import io.airlift.concurrent.SetThreadName;
import io.airlift.log.Logger;
import io.airlift.stats.TimeStat;
import io.airlift.units.Duration;
import io.trino.Session;
import io.trino.exchange.ExchangeInput;
import io.trino.exchange.SpoolingExchangeInput;
import io.trino.execution.BasicStageStats;
import io.trino.execution.NodeTaskMap;
Expand All @@ -42,6 +41,7 @@
import io.trino.spi.exchange.ExchangeId;
import io.trino.spi.exchange.ExchangeManager;
import io.trino.spi.exchange.ExchangeSourceHandle;
import io.trino.spi.exchange.ExchangeSourceOutputSelector;
import io.trino.sql.planner.NodePartitioningManager;
import io.trino.sql.planner.PlanFragment;
import io.trino.sql.planner.SubPlan;
Expand All @@ -62,7 +62,6 @@
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Ticker.systemTicker;
import static com.google.common.base.Verify.verify;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static com.google.common.collect.Lists.reverse;
import static io.airlift.concurrent.MoreFutures.addExceptionCallback;
import static io.airlift.concurrent.MoreFutures.addSuccessCallback;
Expand All @@ -71,6 +70,7 @@
import static io.trino.SystemSessionProperties.getFaultTolerantExecutionPartitionCount;
import static io.trino.SystemSessionProperties.getRetryPolicy;
import static io.trino.execution.QueryState.FINISHING;
import static io.trino.execution.scheduler.Exchanges.getAllSourceHandles;
import static io.trino.operator.RetryPolicy.TASK;
import static java.util.Objects.requireNonNull;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
Expand Down Expand Up @@ -214,7 +214,7 @@ private Scheduler createScheduler()
session,
getFaultTolerantExecutionPartitionCount(session));

ImmutableList.Builder<FaultTolerantStageScheduler> schedulers = ImmutableList.builder();
Map<PlanFragmentId, FaultTolerantStageScheduler> schedulers = new HashMap<>();
Map<PlanFragmentId, Exchange> exchanges = new HashMap<>();
NodeAllocator nodeAllocator = nodeAllocatorService.getNodeAllocator(session);

Expand All @@ -226,7 +226,6 @@ private Scheduler createScheduler()

checkArgument(taskRetryAttemptsOverall >= 0, "taskRetryAttemptsOverall must be greater than or equal to 0: %s", taskRetryAttemptsOverall);
AtomicInteger remainingTaskRetryAttemptsOverall = new AtomicInteger(taskRetryAttemptsOverall);
List<Exchange> outputStages = new ArrayList<>();
for (SqlStage stage : stagesInReverseTopologicalOrder) {
PlanFragment fragment = stage.getFragment();

Expand All @@ -241,17 +240,16 @@ private Scheduler createScheduler()
outputStage);
exchanges.put(fragment.getId(), exchange);

if (outputStage) {
// output will be consumed by coordinator
outputStages.add(exchange);
}

ImmutableMap.Builder<PlanFragmentId, Exchange> sourceExchanges = ImmutableMap.builder();
ImmutableMap.Builder<PlanFragmentId, FaultTolerantStageScheduler> sourceSchedulers = ImmutableMap.builder();
for (SqlStage childStage : stageManager.getChildren(fragment.getId())) {
PlanFragmentId childFragmentId = childStage.getFragment().getId();
Exchange sourceExchange = exchanges.get(childFragmentId);
verify(sourceExchange != null, "exchange not found for fragment: %s", childFragmentId);
sourceExchanges.put(childFragmentId, sourceExchange);
FaultTolerantStageScheduler sourceScheduler = schedulers.get(childFragmentId);
verify(sourceScheduler != null, "scheduler not found for fragment: %s", childFragmentId);
sourceSchedulers.put(childFragmentId, sourceScheduler);
}

FaultTolerantPartitioningScheme sourcePartitioningScheme = partitioningSchemeFactory.get(fragment.getPartitioning());
Expand All @@ -268,45 +266,46 @@ private Scheduler createScheduler()
systemTicker(),
exchange,
sinkPartitioningScheme,
sourceSchedulers.buildOrThrow(),
sourceExchanges.buildOrThrow(),
sourcePartitioningScheme,
remainingTaskRetryAttemptsOverall,
taskRetryAttemptsPerTask,
maxTasksWaitingForNodePerStage,
dynamicFilterService);

schedulers.add(scheduler);
}
schedulers.put(fragment.getId(), scheduler);

if (!stagesInReverseTopologicalOrder.isEmpty()) {
verify(!outputStages.isEmpty(), "coordinatorConsumedExchanges is empty");
List<ListenableFuture<List<ExchangeSourceHandle>>> futures = outputStages.stream()
.map(Exchange::getSourceHandles)
.map(Exchanges::getAllSourceHandles)
.collect(toImmutableList());
ListenableFuture<List<List<ExchangeSourceHandle>>> allFuture = Futures.allAsList(futures);
addSuccessCallback(allFuture, result -> {
List<ExchangeSourceHandle> handles = result.stream()
.flatMap(List::stream)
.collect(toImmutableList());
ImmutableList.Builder<ExchangeInput> inputs = ImmutableList.builder();
if (!handles.isEmpty()) {
inputs.add(new SpoolingExchangeInput(handles));
}
queryStateMachine.updateInputsForQueryResults(inputs.build(), true);
});
addExceptionCallback(allFuture, queryStateMachine::transitionToFailed);
if (outputStage) {
ListenableFuture<List<ExchangeSourceHandle>> sourceHandles = getAllSourceHandles(exchange.getSourceHandles());
addSuccessCallback(sourceHandles, handles -> {
try {
ExchangeSourceOutputSelector.Builder selector = ExchangeSourceOutputSelector.builder(ImmutableSet.of(exchange.getId()));
Map<Integer, Integer> successfulAttempts = scheduler.getSuccessfulAttempts();
successfulAttempts.forEach((taskPartitionId, attemptId) ->
selector.include(exchange.getId(), taskPartitionId, attemptId));
selector.setPartitionCount(exchange.getId(), successfulAttempts.size());
selector.setFinal();
SpoolingExchangeInput input = new SpoolingExchangeInput(handles, Optional.of(selector.build()));
queryStateMachine.updateInputsForQueryResults(ImmutableList.of(input), true);
}
catch (Throwable t) {
queryStateMachine.transitionToFailed(t);
}
});
addExceptionCallback(sourceHandles, queryStateMachine::transitionToFailed);
}
}

return new Scheduler(
queryStateMachine,
schedulers.build(),
ImmutableList.copyOf(schedulers.values()),
stageManager,
schedulerStats,
nodeAllocator);
}
catch (Throwable t) {
for (FaultTolerantStageScheduler scheduler : schedulers.build()) {
for (FaultTolerantStageScheduler scheduler : schedulers.values()) {
try {
scheduler.abort();
}
Expand Down
Loading