Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Add diagnostics for stuck FTE scheduler #19879

Merged
merged 7 commits into from
Nov 28, 2023
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
14 changes: 12 additions & 2 deletions core/trino-main/src/main/java/io/trino/execution/SqlStage.java
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static io.trino.SystemSessionProperties.isEnableCoordinatorDynamicFiltersDistribution;
Expand Down Expand Up @@ -329,9 +330,18 @@ private void checkAllTaskFinal()
}

@Override
public String toString()
// for debugging
public synchronized String toString()
{
return stateMachine.toString();
return toStringHelper(this)
.add("stateMachine", stateMachine)
.add("summarizeTaskInfo", summarizeTaskInfo)
.add("outboundDynamicFilterIds", outboundDynamicFilterIds)
.add("tasks", tasks)
.add("allTasks", allTasks)
.add("finishedTasks", finishedTasks)
.add("tasksWithFinalInfo", tasksWithFinalInfo)
.toString();
}

private class MemoryUsageListener
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import io.trino.spi.exchange.ExchangeSinkInstanceHandle;

import static com.google.common.base.MoreObjects.toStringHelper;
import static com.google.common.base.Preconditions.checkArgument;
import static java.util.Objects.requireNonNull;

Expand Down Expand Up @@ -76,4 +77,13 @@ public SpoolingOutputBuffers withExchangeSinkInstanceHandle(ExchangeSinkInstance
{
return new SpoolingOutputBuffers(getVersion() + 1, handle, outputPartitionCount);
}

@Override
public String toString()
{
return toStringHelper(this)
.add("exchangeSinkInstanceHandle", exchangeSinkInstanceHandle)
.add("outputPartitionCount", outputPartitionCount)
.toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import java.util.Optional;
import java.util.Set;

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.base.Verify.verify;
Expand Down Expand Up @@ -314,6 +315,32 @@ private AssignmentResult assignPartitionedSplits(PlanNodeId planNodeId, List<Spl
return assignment.build();
}

@Override
public String toString()
{
return toStringHelper(this)
.add("catalogRequirement", catalogRequirement)
.add("partitionedSources", partitionedSources)
.add("replicatedSources", replicatedSources)
.add("allSources", allSources)
.add("adaptiveGrowthPeriod", adaptiveGrowthPeriod)
.add("adaptiveGrowthFactor", adaptiveGrowthFactor)
.add("minTargetPartitionSizeInBytes", minTargetPartitionSizeInBytes)
.add("maxTargetPartitionSizeInBytes", maxTargetPartitionSizeInBytes)
.add("standardSplitSizeInBytes", standardSplitSizeInBytes)
.add("maxTaskSplitCount", maxTaskSplitCount)
.add("nextPartitionId", nextPartitionId)
.add("adaptiveCounter", adaptiveCounter)
.add("targetPartitionSizeInBytes", targetPartitionSizeInBytes)
.add("roundedTargetPartitionSizeInBytes", roundedTargetPartitionSizeInBytes)
.add("allAssignments", allAssignments)
.add("openAssignments", openAssignments)
.add("completedSources", completedSources)
.add("replicatedSplits.size()", replicatedSplits.size())
.add("noMoreReplicatedSplits", noMoreReplicatedSplits)
.toString();
}

private Optional<HostAddress> getHostRequirement(Split split)
{
if (split.getConnectorSplit().isRemotelyAccessible()) {
Expand Down Expand Up @@ -396,5 +423,16 @@ public void setFull(boolean full)
{
this.full = full;
}

@Override
public String toString()
{
return toStringHelper(this)
.add("partitionId", partitionId)
.add("assignedDataSizeInBytes", assignedDataSizeInBytes)
.add("assignedSplitCount", assignedSplitCount)
.add("full", full)
.toString();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,7 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;

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.ImmutableList.toImmutableList;
Expand Down Expand Up @@ -459,6 +460,18 @@ public void release()
throw new IllegalStateException("Node " + node + " already released");
}
}

@Override
public String toString()
{
return toStringHelper(this)
.add("node", node)
.add("released", released)
.add("memoryLease", memoryLease)
.add("taskId", taskId)
.add("executionClass", executionClass)
.toString();
}
}

private static class BinPackingSimulation
Expand Down
Loading