diff --git a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java index 66514000ecda..583ae7560f22 100644 --- a/core/trino-main/src/main/java/io/trino/FeaturesConfig.java +++ b/core/trino-main/src/main/java/io/trino/FeaturesConfig.java @@ -68,9 +68,6 @@ public class FeaturesConfig static final String SPILL_ENABLED = "spill-enabled"; public static final String SPILLER_SPILL_PATH = "spiller-spill-path"; - private boolean groupedExecutionEnabled; - private boolean dynamicScheduleForGroupedExecution; - private int concurrentLifespansPerTask; private boolean redistributeWrites = true; private boolean scaleWriters = true; private DataSize writerMinSize = DataSize.of(32, DataSize.Unit.MEGABYTE); @@ -137,47 +134,6 @@ public FeaturesConfig setLegacyRowToJsonCast(boolean legacyRowToJsonCast) return this; } - public boolean isGroupedExecutionEnabled() - { - return groupedExecutionEnabled; - } - - @Config("grouped-execution-enabled") - @ConfigDescription("Experimental: Use grouped execution when possible") - public FeaturesConfig setGroupedExecutionEnabled(boolean groupedExecutionEnabled) - { - this.groupedExecutionEnabled = groupedExecutionEnabled; - return this; - } - - public boolean isDynamicScheduleForGroupedExecutionEnabled() - { - return dynamicScheduleForGroupedExecution; - } - - @Config("dynamic-schedule-for-grouped-execution") - @ConfigDescription("Experimental: Use dynamic schedule for grouped execution when possible") - public FeaturesConfig setDynamicScheduleForGroupedExecutionEnabled(boolean dynamicScheduleForGroupedExecution) - { - this.dynamicScheduleForGroupedExecution = dynamicScheduleForGroupedExecution; - return this; - } - - @Min(0) - public int getConcurrentLifespansPerTask() - { - return concurrentLifespansPerTask; - } - - @Config("concurrent-lifespans-per-task") - @ConfigDescription("Experimental: Default number of lifespans that run in parallel on each task when grouped execution is enabled") - // When set to zero, a limit is not imposed on the number of lifespans that run in parallel - public FeaturesConfig setConcurrentLifespansPerTask(int concurrentLifespansPerTask) - { - this.concurrentLifespansPerTask = concurrentLifespansPerTask; - return this; - } - public boolean isRedistributeWrites() { return redistributeWrites; diff --git a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java index 65bdd3d26e06..fd1201500d4b 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -39,7 +39,6 @@ import static io.trino.plugin.base.session.PropertyMetadataUtil.dataSizeProperty; import static io.trino.plugin.base.session.PropertyMetadataUtil.durationProperty; import static io.trino.spi.StandardErrorCode.INVALID_SESSION_PROPERTY; -import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.session.PropertyMetadata.booleanProperty; import static io.trino.spi.session.PropertyMetadata.doubleProperty; import static io.trino.spi.session.PropertyMetadata.enumProperty; @@ -61,8 +60,6 @@ public final class SystemSessionProperties public static final String JOIN_MULTI_CLAUSE_INDEPENDENCE_FACTOR = "join_multi_clause_independence_factor"; public static final String DISTRIBUTED_INDEX_JOIN = "distributed_index_join"; public static final String HASH_PARTITION_COUNT = "hash_partition_count"; - public static final String GROUPED_EXECUTION = "grouped_execution"; - public static final String DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION = "dynamic_schedule_for_grouped_execution"; public static final String PREFER_STREAMING_OPERATORS = "prefer_streaming_operators"; public static final String TASK_WRITER_COUNT = "task_writer_count"; public static final String TASK_CONCURRENCY = "task_concurrency"; @@ -89,7 +86,6 @@ public final class SystemSessionProperties public static final String SPATIAL_JOIN = "spatial_join"; public static final String SPATIAL_PARTITIONING_TABLE_NAME = "spatial_partitioning_table_name"; public static final String COLOCATED_JOIN = "colocated_join"; - public static final String CONCURRENT_LIFESPANS_PER_NODE = "concurrent_lifespans_per_task"; public static final String JOIN_REORDERING_STRATEGY = "join_reordering_strategy"; public static final String MAX_REORDERED_JOINS = "max_reordered_joins"; public static final String INITIAL_SPLITS_PER_NODE = "initial_splits_per_node"; @@ -240,16 +236,6 @@ public SystemSessionProperties( "Number of partitions for distributed joins and aggregations", queryManagerConfig.getHashPartitionCount(), false), - booleanProperty( - GROUPED_EXECUTION, - "Use grouped execution when possible", - featuresConfig.isGroupedExecutionEnabled(), - false), - booleanProperty( - DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, - "Experimental: Use dynamic schedule for grouped execution when possible", - featuresConfig.isDynamicScheduleForGroupedExecutionEnabled(), - false), booleanProperty( PREFER_STREAMING_OPERATORS, "Prefer source table layouts that produce streaming operators", @@ -423,11 +409,6 @@ public SystemSessionProperties( "Name of the table containing spatial partitioning scheme", null, false), - integerProperty( - CONCURRENT_LIFESPANS_PER_NODE, - "Experimental: Run a fixed number of groups concurrently for eligible JOINs", - featuresConfig.getConcurrentLifespansPerTask(), - false), booleanProperty( SPILL_ENABLED, "Enable spilling", @@ -886,16 +867,6 @@ public static int getHashPartitionCount(Session session) return session.getSystemProperty(HASH_PARTITION_COUNT, Integer.class); } - public static boolean isGroupedExecutionEnabled(Session session) - { - return session.getSystemProperty(GROUPED_EXECUTION, Boolean.class); - } - - public static boolean isDynamicScheduleForGroupedExecution(Session session) - { - return session.getSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, Boolean.class); - } - public static boolean preferStreamingOperators(Session session) { return session.getSystemProperty(PREFER_STREAMING_OPERATORS, Boolean.class); @@ -1026,16 +997,6 @@ public static Optional getSpatialPartitioningTableName(Session session) return Optional.ofNullable(session.getSystemProperty(SPATIAL_PARTITIONING_TABLE_NAME, String.class)); } - public static OptionalInt getConcurrentLifespansPerNode(Session session) - { - Integer result = session.getSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, Integer.class); - if (result == 0) { - return OptionalInt.empty(); - } - checkArgument(result > 0, "Concurrent lifespans per node is negative: %s", result); - return OptionalInt.of(result); - } - public static int getInitialSplitsPerNode(Session session) { return session.getSystemProperty(INITIAL_SPLITS_PER_NODE, Integer.class); @@ -1402,13 +1363,7 @@ public static DataSize getMaxPartialTopNMemory(Session session) public static RetryPolicy getRetryPolicy(Session session) { - RetryPolicy retryPolicy = session.getSystemProperty(RETRY_POLICY, RetryPolicy.class); - 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; + return session.getSystemProperty(RETRY_POLICY, RetryPolicy.class); } public static int getQueryRetryAttempts(Session session) diff --git a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java index 3116dabe8a2b..0924eff62137 100644 --- a/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/informationschema/InformationSchemaSplitManager.java @@ -45,7 +45,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java b/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java index c4e6550e965c..fcfa27caaba6 100644 --- a/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java +++ b/core/trino-main/src/main/java/io/trino/connector/system/SystemSplitManager.java @@ -58,7 +58,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/core/trino-main/src/main/java/io/trino/execution/Lifespan.java b/core/trino-main/src/main/java/io/trino/execution/Lifespan.java deleted file mode 100644 index 8be2f53d77aa..000000000000 --- a/core/trino-main/src/main/java/io/trino/execution/Lifespan.java +++ /dev/null @@ -1,104 +0,0 @@ -/* - * 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.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; -import org.openjdk.jol.info.ClassLayout; - -import java.util.Objects; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; -import static java.lang.Integer.parseInt; - -public class Lifespan -{ - private static final int INSTANCE_SIZE = ClassLayout.parseClass(Lifespan.class).instanceSize(); - - private static final Lifespan TASK_WIDE = new Lifespan(false, 0); - - private final boolean grouped; - private final int groupId; - - public static Lifespan taskWide() - { - return TASK_WIDE; - } - - public static Lifespan driverGroup(int id) - { - return new Lifespan(true, id); - } - - private Lifespan(boolean grouped, int groupId) - { - this.grouped = grouped; - this.groupId = groupId; - } - - public boolean isTaskWide() - { - return !grouped; - } - - public int getId() - { - checkState(grouped); - return groupId; - } - - @JsonCreator - public static Lifespan jsonCreator(String value) - { - if (value.equals("TaskWide")) { - return Lifespan.taskWide(); - } - checkArgument(value.startsWith("Group")); - return Lifespan.driverGroup(parseInt(value.substring("Group".length()))); - } - - @Override - @JsonValue - public String toString() - { - return grouped ? "Group" + groupId : "TaskWide"; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Lifespan that = (Lifespan) o; - return grouped == that.grouped && - groupId == that.groupId; - } - - @Override - public int hashCode() - { - return Objects.hash(grouped, groupId); - } - - public long getRetainedSizeInBytes() - { - return INSTANCE_SIZE; - } -} diff --git a/core/trino-main/src/main/java/io/trino/execution/RemoteTask.java b/core/trino-main/src/main/java/io/trino/execution/RemoteTask.java index c72efae9aa79..72056023cda4 100644 --- a/core/trino-main/src/main/java/io/trino/execution/RemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/RemoteTask.java @@ -36,8 +36,6 @@ public interface RemoteTask void noMoreSplits(PlanNodeId sourceId); - void noMoreSplits(PlanNodeId sourceId, Lifespan lifespan); - void setOutputBuffers(OutputBuffers outputBuffers); /** diff --git a/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java b/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java index 2f77bcb821f0..d2b7cf250760 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java +++ b/core/trino-main/src/main/java/io/trino/execution/SplitAssignment.java @@ -28,27 +28,19 @@ public class SplitAssignment { private final PlanNodeId planNodeId; private final Set splits; - private final Set noMoreSplitsForLifespan; private final boolean noMoreSplits; @JsonCreator public SplitAssignment( @JsonProperty("planNodeId") PlanNodeId planNodeId, @JsonProperty("splits") Set splits, - @JsonProperty("noMoreSplitsForLifespan") Set noMoreSplitsForLifespan, @JsonProperty("noMoreSplits") boolean noMoreSplits) { this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); this.splits = ImmutableSet.copyOf(requireNonNull(splits, "splits is null")); - this.noMoreSplitsForLifespan = ImmutableSet.copyOf(noMoreSplitsForLifespan); this.noMoreSplits = noMoreSplits; } - public SplitAssignment(PlanNodeId planNodeId, Set splits, boolean noMoreSplits) - { - this(planNodeId, splits, ImmutableSet.of(), noMoreSplits); - } - @JsonProperty public PlanNodeId getPlanNodeId() { @@ -61,12 +53,6 @@ public Set getSplits() return splits; } - @JsonProperty - public Set getNoMoreSplitsForLifespan() - { - return noMoreSplitsForLifespan; - } - @JsonProperty public boolean isNoMoreSplits() { @@ -86,15 +72,10 @@ public SplitAssignment update(SplitAssignment assignment) .addAll(splits) .addAll(assignment.getSplits()) .build(); - Set newNoMoreSplitsForDriverGroup = ImmutableSet.builder() - .addAll(noMoreSplitsForLifespan) - .addAll(assignment.getNoMoreSplitsForLifespan()) - .build(); return new SplitAssignment( planNodeId, newSplits, - newNoMoreSplitsForDriverGroup, assignment.isNoMoreSplits()); } else { @@ -108,7 +89,6 @@ private boolean isNewer(SplitAssignment assignment) // the specified assignment is newer if it changes the no more // splits flag or if it contains new splits return (!noMoreSplits && assignment.isNoMoreSplits()) || - (!noMoreSplitsForLifespan.containsAll(assignment.getNoMoreSplitsForLifespan())) || (!splits.containsAll(assignment.getSplits())); } diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlStage.java b/core/trino-main/src/main/java/io/trino/execution/SqlStage.java index 2c8dfa3cc22f..6a8568d9ed44 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlStage.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlStage.java @@ -213,7 +213,6 @@ public synchronized Optional createTask( Optional bucketToPartition, OutputBuffers outputBuffers, Multimap splits, - Multimap noMoreSplitsForLifespan, Set noMoreSplits, Optional estimatedMemory) { @@ -237,7 +236,6 @@ public synchronized Optional createTask( estimatedMemory, summarizeTaskInfo); - noMoreSplitsForLifespan.forEach(task::noMoreSplits); noMoreSplits.forEach(task::noMoreSplits); tasks.put(taskId, task); diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java index 4051e7beeb3f..acf3ed9e8f58 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlTask.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlTask.java @@ -292,8 +292,6 @@ private TaskStatus createTaskStatus(TaskHolder taskHolder) DataSize userMemoryReservation = DataSize.ofBytes(0); DataSize peakUserMemoryReservation = DataSize.ofBytes(0); DataSize revocableMemoryReservation = DataSize.ofBytes(0); - // TODO: add a mechanism to avoid sending the whole completedDriverGroups set over the wire for every task status reply - Set completedDriverGroups = ImmutableSet.of(); long fullGcCount = 0; Duration fullGcTime = new Duration(0, MILLISECONDS); long dynamicFiltersVersion = INITIAL_DYNAMIC_FILTERS_VERSION; @@ -325,7 +323,6 @@ else if (taskHolder.getTaskExecution() != null) { physicalWrittenDataSize = succinctBytes(physicalWrittenBytes); userMemoryReservation = taskContext.getMemoryReservation(); revocableMemoryReservation = taskContext.getRevocableMemoryReservation(); - completedDriverGroups = taskContext.getCompletedDriverGroups(); fullGcCount = taskContext.getFullGcCount(); fullGcTime = taskContext.getFullGcTime(); dynamicFiltersVersion = taskContext.getDynamicFiltersVersion(); @@ -337,7 +334,6 @@ else if (taskHolder.getTaskExecution() != null) { state, location, nodeId, - completedDriverGroups, failures, queuedPartitionedDrivers, runningPartitionedDrivers, diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java index 18680cd7e2cf..d06dc0f62164 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecution.java @@ -13,7 +13,6 @@ */ package io.trino.execution; -import com.google.common.collect.AbstractIterator; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -34,8 +33,6 @@ import io.trino.operator.DriverFactory; import io.trino.operator.DriverStats; import io.trino.operator.PipelineContext; -import io.trino.operator.PipelineExecutionStrategy; -import io.trino.operator.StageExecutionDescriptor; import io.trino.operator.TaskContext; import io.trino.spi.SplitWeight; import io.trino.spi.TrinoException; @@ -51,7 +48,6 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -69,6 +65,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verify; import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.util.concurrent.Futures.immediateVoidFuture; import static io.trino.SystemSessionProperties.getInitialSplitsPerNode; import static io.trino.SystemSessionProperties.getMaxDriversPerTask; @@ -76,7 +73,6 @@ import static io.trino.execution.SqlTaskExecution.SplitsState.ADDING_SPLITS; import static io.trino.execution.SqlTaskExecution.SplitsState.FINISHED; import static io.trino.execution.SqlTaskExecution.SplitsState.NO_MORE_SPLITS; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -85,33 +81,6 @@ public class SqlTaskExecution { - // For each driver in a task, it belong to a pipeline and a driver life cycle. - // Pipeline and driver life cycle are two perpendicular organizations of tasks. - // - // * All drivers in the same pipeline has the same shape. - // (i.e. operators are constructed from the same set of operator factories) - // * All drivers in the same driver life cycle are responsible for processing a group of data. - // (e.g. all rows that fall in bucket 42) - // - // Take a task with the following set of pipelines for example: - // - // pipeline 0 pipeline 1 pipeline 2 pipeline 3 ... pipeline id - // grouped grouped grouped ungrouped ... execution strategy - // - // PartitionedOutput - // | - // LookupJoin .................................. HashBuild - // | | - // LookupJoin ... HashBuild ExchangeSrc - // | | - // TableScan LocalExSrc ... LocalExSink - // | - // TableScan - // - // In this case, - // * a driver could belong to pipeline 1 and driver life cycle 42. - // * another driver could belong to pipeline 3 and task-wide driver life cycle. - private final TaskId taskId; private final TaskStateMachine taskStateMachine; private final TaskContext taskContext; @@ -127,7 +96,6 @@ public class SqlTaskExecution private final List> drivers = new CopyOnWriteArrayList<>(); private final Map driverRunnerFactoriesWithSplitLifeCycle; - private final List driverRunnerFactoriesWithDriverGroupLifeCycle; private final List driverRunnerFactoriesWithTaskLifeCycle; // guarded for update only @@ -138,7 +106,9 @@ public class SqlTaskExecution private long maxAcknowledgedSplit = Long.MIN_VALUE; @GuardedBy("this") - private final SchedulingLifespanManager schedulingLifespanManager; + private final List sourceStartOrder; + @GuardedBy("this") + private int schedulingPlanNodeOrdinal; @GuardedBy("this") private final Map pendingSplitsByPlanNode; @@ -194,50 +164,29 @@ private SqlTaskExecution( Set partitionedSources = ImmutableSet.copyOf(localExecutionPlan.getPartitionedSourceOrder()); ImmutableMap.Builder driverRunnerFactoriesWithSplitLifeCycle = ImmutableMap.builder(); ImmutableList.Builder driverRunnerFactoriesWithTaskLifeCycle = ImmutableList.builder(); - ImmutableList.Builder driverRunnerFactoriesWithDriverGroupLifeCycle = ImmutableList.builder(); for (DriverFactory driverFactory : localExecutionPlan.getDriverFactories()) { Optional sourceId = driverFactory.getSourceId(); if (sourceId.isPresent() && partitionedSources.contains(sourceId.get())) { driverRunnerFactoriesWithSplitLifeCycle.put(sourceId.get(), new DriverSplitRunnerFactory(driverFactory, true)); } else { - switch (driverFactory.getPipelineExecutionStrategy()) { - case GROUPED_EXECUTION: - driverRunnerFactoriesWithDriverGroupLifeCycle.add(new DriverSplitRunnerFactory(driverFactory, false)); - break; - case UNGROUPED_EXECUTION: - driverRunnerFactoriesWithTaskLifeCycle.add(new DriverSplitRunnerFactory(driverFactory, false)); - break; - default: - throw new UnsupportedOperationException(); - } + driverRunnerFactoriesWithTaskLifeCycle.add(new DriverSplitRunnerFactory(driverFactory, false)); } } this.driverRunnerFactoriesWithSplitLifeCycle = driverRunnerFactoriesWithSplitLifeCycle.buildOrThrow(); - this.driverRunnerFactoriesWithDriverGroupLifeCycle = driverRunnerFactoriesWithDriverGroupLifeCycle.build(); this.driverRunnerFactoriesWithTaskLifeCycle = driverRunnerFactoriesWithTaskLifeCycle.build(); this.pendingSplitsByPlanNode = this.driverRunnerFactoriesWithSplitLifeCycle.keySet().stream() .collect(toImmutableMap(identity(), ignore -> new PendingSplitsForPlanNode())); this.status = new Status( - taskContext, localExecutionPlan.getDriverFactories().stream() - .collect(toImmutableMap(DriverFactory::getPipelineId, DriverFactory::getPipelineExecutionStrategy))); - this.schedulingLifespanManager = new SchedulingLifespanManager(localExecutionPlan.getPartitionedSourceOrder(), localExecutionPlan.getStageExecutionDescriptor(), this.status); + .map(DriverFactory::getPipelineId) + .collect(toImmutableSet())); + sourceStartOrder = localExecutionPlan.getPartitionedSourceOrder(); checkArgument(this.driverRunnerFactoriesWithSplitLifeCycle.keySet().equals(partitionedSources), "Fragment is partitioned, but not all partitioned drivers were found"); - // Pre-register Lifespans for ungrouped partitioned drivers in case they end up get no splits. - for (Entry entry : this.driverRunnerFactoriesWithSplitLifeCycle.entrySet()) { - PlanNodeId planNodeId = entry.getKey(); - DriverSplitRunnerFactory driverSplitRunnerFactory = entry.getValue(); - if (driverSplitRunnerFactory.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION) { - this.schedulingLifespanManager.addLifespanIfAbsent(Lifespan.taskWide()); - this.pendingSplitsByPlanNode.get(planNodeId).getLifespan(Lifespan.taskWide()); - } - } - // don't register the task if it is already completed (most likely failed during planning above) if (!taskStateMachine.getState().isDone()) { taskHandle = createTaskHandle(taskStateMachine, taskContext, outputBuffer, localExecutionPlan, taskExecutor); @@ -334,9 +283,6 @@ private synchronized Map updateSplitAssignments(Lis source.getSplits().stream() .filter(scheduledSplit -> scheduledSplit.getSequenceId() > currentMaxAcknowledgedSplit) .collect(Collectors.toSet()), - // Like splits, noMoreSplitsForLifespan could be pruned so that only new items will be processed. - // This is not happening here because correctness won't be compromised due to duplicate events for noMoreSplitsForLifespan. - source.getNoMoreSplitsForLifespan(), source.isNoMoreSplits())) .collect(toList()); @@ -351,7 +297,7 @@ private synchronized Map updateSplitAssignments(Lis } for (DriverSplitRunnerFactory driverSplitRunnerFactory : - Iterables.concat(driverRunnerFactoriesWithSplitLifeCycle.values(), driverRunnerFactoriesWithTaskLifeCycle, driverRunnerFactoriesWithDriverGroupLifeCycle)) { + Iterables.concat(driverRunnerFactoriesWithSplitLifeCycle.values(), driverRunnerFactoriesWithTaskLifeCycle)) { driverSplitRunnerFactory.closeDriverFactoryIfFullyCreated(); } @@ -365,7 +311,7 @@ private synchronized Map updateSplitAssignments(Lis } @GuardedBy("this") - private void mergeIntoPendingSplits(PlanNodeId planNodeId, Set scheduledSplits, Set noMoreSplitsForLifespan, boolean noMoreSplits) + private void mergeIntoPendingSplits(PlanNodeId planNodeId, Set scheduledSplits, boolean noMoreSplits) { checkHoldsLock(); @@ -374,15 +320,7 @@ private void mergeIntoPendingSplits(PlanNodeId planNodeId, Set s partitionedDriverFactory.splitsAdded(scheduledSplits.size(), SplitWeight.rawValueSum(scheduledSplits, scheduledSplit -> scheduledSplit.getSplit().getSplitWeight())); for (ScheduledSplit scheduledSplit : scheduledSplits) { - Lifespan lifespan = scheduledSplit.getSplit().getLifespan(); - checkLifespan(partitionedDriverFactory.getPipelineExecutionStrategy(), lifespan); - pendingSplitsForPlanNode.getLifespan(lifespan).addSplit(scheduledSplit); - schedulingLifespanManager.addLifespanIfAbsent(lifespan); - } - for (Lifespan lifespanWithNoMoreSplits : noMoreSplitsForLifespan) { - checkLifespan(partitionedDriverFactory.getPipelineExecutionStrategy(), lifespanWithNoMoreSplits); - pendingSplitsForPlanNode.getLifespan(lifespanWithNoMoreSplits).noMoreSplits(); - schedulingLifespanManager.addLifespanIfAbsent(lifespanWithNoMoreSplits); + pendingSplitsForPlanNode.addSplit(scheduledSplit); } if (noMoreSplits) { pendingSplitsForPlanNode.setNoMoreSplits(); @@ -391,96 +329,32 @@ private void mergeIntoPendingSplits(PlanNodeId planNodeId, Set s private synchronized void schedulePartitionedSource(SplitAssignment splitAssignmentUpdate) { - mergeIntoPendingSplits(splitAssignmentUpdate.getPlanNodeId(), splitAssignmentUpdate.getSplits(), splitAssignmentUpdate.getNoMoreSplitsForLifespan(), splitAssignmentUpdate.isNoMoreSplits()); - - while (true) { - // SchedulingLifespanManager tracks how far each Lifespan has been scheduled. Here is an example. - // Let's say there are 4 source pipelines/nodes: A, B, C, and D, in scheduling order. - // And we're processing 3 concurrent lifespans at a time. In this case, we could have - // - // * Lifespan 10: A B [C] D; i.e. Pipeline A and B has finished scheduling (but not necessarily finished running). - // * Lifespan 20: [A] B C D - // * Lifespan 30: A [B] C D - // - // To recap, SchedulingLifespanManager records the next scheduling source node for each lifespan. - Iterator activeLifespans = schedulingLifespanManager.getActiveLifespans(); - - boolean madeProgress = false; - - while (activeLifespans.hasNext()) { - SchedulingLifespan schedulingLifespan = activeLifespans.next(); - Lifespan lifespan = schedulingLifespan.getLifespan(); - - // Continue using the example from above. Let's say the splitAssignmentUpdate adds some new splits for source node B. - // - // For lifespan 30, it could start new drivers and assign a pending split to each. - // Pending splits could include both pre-existing pending splits, and the new ones from splitAssignmentUpdate. - // If there is enough driver slots to deplete pending splits, one of the below would happen. - // * If it is marked that all splits for node B in lifespan 30 has been received, SchedulingLifespanManager - // will be updated so that lifespan 30 now processes source node C. It will immediately start processing them. - // * Otherwise, processing of lifespan 30 will be shelved for now. - // - // It is possible that the following loop would be a no-op for a particular lifespan. - // It is also possible that a single lifespan can proceed through multiple source nodes in one run. - // - // When different drivers in the task has different pipelineExecutionStrategy, it adds additional complexity. - // For example, when driver B is ungrouped and driver A, C, D is grouped, you could have something like this: - // TaskWide : [B] - // Lifespan 10: A [ ] C D - // Lifespan 20: [A] C D - // Lifespan 30: A [ ] C D - // In this example, Lifespan 30 cannot start executing drivers in pipeline C because pipeline B - // hasn't finished scheduling yet (albeit in a different lifespan). - // Similarly, it wouldn't make sense for TaskWide to start executing drivers in pipeline B until at least - // one lifespan has finished scheduling pipeline A. - // This is why getSchedulingPlanNode returns an Optional. - while (true) { - Optional optionalSchedulingPlanNode = schedulingLifespan.getSchedulingPlanNode(); - if (optionalSchedulingPlanNode.isEmpty()) { - break; - } - PlanNodeId schedulingPlanNode = optionalSchedulingPlanNode.get(); - - DriverSplitRunnerFactory partitionedDriverRunnerFactory = driverRunnerFactoriesWithSplitLifeCycle.get(schedulingPlanNode); - - PendingSplits pendingSplits = pendingSplitsByPlanNode.get(schedulingPlanNode).getLifespan(lifespan); + mergeIntoPendingSplits(splitAssignmentUpdate.getPlanNodeId(), splitAssignmentUpdate.getSplits(), splitAssignmentUpdate.isNoMoreSplits()); - // Enqueue driver runners with driver group lifecycle for this driver life cycle, if not already enqueued. - if (!lifespan.isTaskWide() && !schedulingLifespan.getAndSetDriversForDriverGroupLifeCycleScheduled()) { - scheduleDriversForDriverGroupLifeCycle(lifespan); - } + while (schedulingPlanNodeOrdinal < sourceStartOrder.size()) { + PlanNodeId schedulingPlanNode = sourceStartOrder.get(schedulingPlanNodeOrdinal); - // Enqueue driver runners with split lifecycle for this plan node and driver life cycle combination. - ImmutableList.Builder runners = ImmutableList.builder(); - for (ScheduledSplit scheduledSplit : pendingSplits.removeAllSplits()) { - // create a new driver for the split - runners.add(partitionedDriverRunnerFactory.createDriverRunner(scheduledSplit, lifespan)); - } - enqueueDriverSplitRunner(false, runners.build()); + DriverSplitRunnerFactory partitionedDriverRunnerFactory = driverRunnerFactoriesWithSplitLifeCycle.get(schedulingPlanNode); - // If all driver runners have been enqueued for this plan node and driver life cycle combination, - // move on to the next plan node. - if (pendingSplits.getState() != NO_MORE_SPLITS) { - break; - } - partitionedDriverRunnerFactory.noMoreDriverRunner(ImmutableList.of(lifespan)); - pendingSplits.markAsCleanedUp(); + PendingSplitsForPlanNode pendingSplits = pendingSplitsByPlanNode.get(schedulingPlanNode); - schedulingLifespan.nextPlanNode(); - madeProgress = true; - if (schedulingLifespan.isDone()) { - break; - } - } + // Enqueue driver runners with split lifecycle for this plan node and driver life cycle combination. + ImmutableList.Builder runners = ImmutableList.builder(); + for (ScheduledSplit scheduledSplit : pendingSplits.removeAllSplits()) { + // create a new driver for the split + runners.add(partitionedDriverRunnerFactory.createDriverRunner(scheduledSplit)); } + enqueueDriverSplitRunner(false, runners.build()); - if (!madeProgress) { + // If all driver runners have been enqueued for this plan node and driver life cycle combination, + // move on to the next plan node. + if (pendingSplits.getState() != NO_MORE_SPLITS) { break; } - } + partitionedDriverRunnerFactory.noMoreDriverRunner(); + pendingSplits.markAsCleanedUp(); - if (splitAssignmentUpdate.isNoMoreSplits()) { - schedulingLifespanManager.noMoreSplits(splitAssignmentUpdate.getPlanNodeId()); + schedulingPlanNodeOrdinal++; } } @@ -513,37 +387,16 @@ private void scheduleDriversForTaskLifeCycle() List runners = new ArrayList<>(); for (DriverSplitRunnerFactory driverRunnerFactory : driverRunnerFactoriesWithTaskLifeCycle) { for (int i = 0; i < driverRunnerFactory.getDriverInstances().orElse(1); i++) { - runners.add(driverRunnerFactory.createDriverRunner(null, Lifespan.taskWide())); + runners.add(driverRunnerFactory.createDriverRunner(null)); } } enqueueDriverSplitRunner(true, runners); for (DriverSplitRunnerFactory driverRunnerFactory : driverRunnerFactoriesWithTaskLifeCycle) { - driverRunnerFactory.noMoreDriverRunner(ImmutableList.of(Lifespan.taskWide())); + driverRunnerFactory.noMoreDriverRunner(); verify(driverRunnerFactory.isNoMoreDriverRunner()); } } - private void scheduleDriversForDriverGroupLifeCycle(Lifespan lifespan) - { - // This method is called when a split that belongs to a previously unseen driver group is scheduled. - // It schedules drivers for all the pipelines that have driver group life cycle. - if (lifespan.isTaskWide()) { - checkArgument(driverRunnerFactoriesWithDriverGroupLifeCycle.isEmpty(), "Instantiating pipeline of driver group lifecycle at task level is not allowed"); - return; - } - - List runners = new ArrayList<>(); - for (DriverSplitRunnerFactory driverSplitRunnerFactory : driverRunnerFactoriesWithDriverGroupLifeCycle) { - for (int i = 0; i < driverSplitRunnerFactory.getDriverInstances().orElse(1); i++) { - runners.add(driverSplitRunnerFactory.createDriverRunner(null, lifespan)); - } - } - enqueueDriverSplitRunner(true, runners); - for (DriverSplitRunnerFactory driverRunnerFactory : driverRunnerFactoriesWithDriverGroupLifeCycle) { - driverRunnerFactory.noMoreDriverRunner(ImmutableList.of(lifespan)); - } - } - private synchronized void enqueueDriverSplitRunner(boolean forceRunSplit, List runners) { // schedule driver to be executed @@ -556,7 +409,7 @@ private synchronized void enqueueDriverSplitRunner(boolean forceRunSplit, List() { @@ -565,7 +418,7 @@ public void onSuccess(Object result) { try (SetThreadName ignored = new SetThreadName("Task-%s", taskId)) { // record driver is finished - status.decrementRemainingDriver(splitRunner.getLifespan()); + status.decrementRemainingDriver(); checkTaskCompletion(); @@ -580,7 +433,7 @@ public void onFailure(Throwable cause) taskStateMachine.failed(cause); // record driver is finished - status.decrementRemainingDriver(splitRunner.getLifespan()); + status.decrementRemainingDriver(); // fire failed event with cause splitMonitor.splitFailedEvent(taskId, getDriverStats(), cause); @@ -675,19 +528,6 @@ public String toString() .toString(); } - private void checkLifespan(PipelineExecutionStrategy executionStrategy, Lifespan lifespan) - { - switch (executionStrategy) { - case GROUPED_EXECUTION: - checkArgument(!lifespan.isTaskWide(), "Expect driver-group life cycle for grouped ExecutionStrategy. Got task-wide life cycle."); - return; - case UNGROUPED_EXECUTION: - checkArgument(lifespan.isTaskWide(), "Expect task-wide life cycle for ungrouped ExecutionStrategy. Got driver-group life cycle."); - return; - } - throw new IllegalArgumentException("Unknown executionStrategy: " + executionStrategy); - } - private void checkHoldsLock() { // This method serves a similar purpose at runtime as GuardedBy on method serves during static analysis. @@ -698,36 +538,24 @@ private void checkHoldsLock() } } - // Splits for a particular plan node (all driver groups) + // Splits for a particular plan node @NotThreadSafe private static class PendingSplitsForPlanNode { - private final Map splitsByLifespan = new HashMap<>(); + private Set splits = new HashSet<>(); + private SplitsState state = ADDING_SPLITS; private boolean noMoreSplits; - public PendingSplits getLifespan(Lifespan lifespan) - { - return splitsByLifespan.computeIfAbsent(lifespan, ignored -> new PendingSplits()); - } - public void setNoMoreSplits() { if (noMoreSplits) { return; } noMoreSplits = true; - for (PendingSplits splitsForLifespan : splitsByLifespan.values()) { - splitsForLifespan.noMoreSplits(); + if (state == ADDING_SPLITS) { + state = NO_MORE_SPLITS; } } - } - - // Splits for a particular plan node and driver group combination - @NotThreadSafe - private static class PendingSplits - { - private Set splits = new HashSet<>(); - private SplitsState state = ADDING_SPLITS; public SplitsState getState() { @@ -748,13 +576,6 @@ public Set removeAllSplits() return result; } - public void noMoreSplits() - { - if (state == ADDING_SPLITS) { - state = NO_MORE_SPLITS; - } - } - public void markAsCleanedUp() { checkState(splits.isEmpty()); @@ -773,161 +594,6 @@ enum SplitsState FINISHED, } - private static class SchedulingLifespanManager - { - // SchedulingLifespanManager only contains partitioned drivers. - // Note that different drivers in a task may have different pipelineExecutionStrategy. - - private final List sourceStartOrder; - private final StageExecutionDescriptor stageExecutionDescriptor; - private final Status status; - - private final Map lifespans = new HashMap<>(); - // driver groups whose scheduling is done (all splits for all plan nodes) - private final Set completedLifespans = new HashSet<>(); - - private final Set noMoreSplits = new HashSet<>(); - - private int maxScheduledPlanNodeOrdinal; - - public SchedulingLifespanManager(List sourceStartOrder, StageExecutionDescriptor stageExecutionDescriptor, Status status) - { - this.sourceStartOrder = ImmutableList.copyOf(sourceStartOrder); - this.stageExecutionDescriptor = stageExecutionDescriptor; - this.status = requireNonNull(status, "status is null"); - } - - public int getMaxScheduledPlanNodeOrdinal() - { - return maxScheduledPlanNodeOrdinal; - } - - public void updateMaxScheduledPlanNodeOrdinalIfNecessary(int scheduledPlanNodeOrdinal) - { - if (maxScheduledPlanNodeOrdinal < scheduledPlanNodeOrdinal) { - maxScheduledPlanNodeOrdinal = scheduledPlanNodeOrdinal; - } - } - - public void noMoreSplits(PlanNodeId planNodeId) - { - if (noMoreSplits.contains(planNodeId)) { - return; - } - noMoreSplits.add(planNodeId); - if (noMoreSplits.size() < sourceStartOrder.size()) { - return; - } - checkState(noMoreSplits.size() == sourceStartOrder.size()); - checkState(noMoreSplits.containsAll(sourceStartOrder)); - status.setNoMoreLifespans(); - } - - public void addLifespanIfAbsent(Lifespan lifespan) - { - if (completedLifespans.contains(lifespan) || lifespans.containsKey(lifespan)) { - return; - } - checkState(!status.isNoMoreLifespans()); - checkState(!sourceStartOrder.isEmpty()); - lifespans.put(lifespan, new SchedulingLifespan(lifespan, this)); - } - - public Iterator getActiveLifespans() - { - // This function returns an iterator that iterates through active driver groups. - // Before it advances to the next item, it checks whether the previous returned driver group is done scheduling. - // If so, the completed SchedulingLifespan is removed so that it will not be returned again. - Iterator lifespansIterator = lifespans.values().iterator(); - return new AbstractIterator<>() - { - SchedulingLifespan lastSchedulingLifespan; - - @Override - protected SchedulingLifespan computeNext() - { - if (lastSchedulingLifespan != null) { - if (lastSchedulingLifespan.isDone()) { - completedLifespans.add(lastSchedulingLifespan.getLifespan()); - lifespansIterator.remove(); - } - } - if (!lifespansIterator.hasNext()) { - return endOfData(); - } - lastSchedulingLifespan = lifespansIterator.next(); - return lastSchedulingLifespan; - } - }; - } - } - - private static class SchedulingLifespan - { - private final Lifespan lifespan; - private final SchedulingLifespanManager manager; - private int schedulingPlanNodeOrdinal; - private boolean unpartitionedDriversScheduled; - - public SchedulingLifespan(Lifespan lifespan, SchedulingLifespanManager manager) - { - this.lifespan = requireNonNull(lifespan, "lifespan is null"); - this.manager = requireNonNull(manager, "manager is null"); - } - - public Lifespan getLifespan() - { - return lifespan; - } - - public Optional getSchedulingPlanNode() - { - checkState(!isDone()); - while (!isDone()) { - // Return current plan node if this lifespan is compatible with the plan node. - // i.e. One of the following bullet points is true: - // * The execution strategy of the plan node is grouped. And lifespan represents a driver group. - // * The execution strategy of the plan node is ungrouped. And lifespan is task wide. - if (manager.stageExecutionDescriptor.isScanGroupedExecution(manager.sourceStartOrder.get(schedulingPlanNodeOrdinal)) != lifespan.isTaskWide()) { - return Optional.of(manager.sourceStartOrder.get(schedulingPlanNodeOrdinal)); - } - // This lifespan is incompatible with the plan node. As a result, this method should either - // return empty to indicate that scheduling for this lifespan is blocked, or skip the current - // plan node and went on to the next one. Which one of the two happens is dependent on whether - // the current plan node has finished scheduling in any other lifespan. - // If so, the lifespan can advance to the next plan node. - // If not, it should not advance because doing so would violate scheduling order. - if (manager.getMaxScheduledPlanNodeOrdinal() == schedulingPlanNodeOrdinal) { - return Optional.empty(); - } - verify(manager.getMaxScheduledPlanNodeOrdinal() > schedulingPlanNodeOrdinal); - nextPlanNode(); - } - return Optional.empty(); - } - - public void nextPlanNode() - { - checkState(!isDone()); - schedulingPlanNodeOrdinal++; - manager.updateMaxScheduledPlanNodeOrdinalIfNecessary(schedulingPlanNodeOrdinal); - } - - public boolean isDone() - { - return schedulingPlanNodeOrdinal >= manager.sourceStartOrder.size(); - } - - public boolean getAndSetDriversForDriverGroupLifeCycleScheduled() - { - if (unpartitionedDriversScheduled) { - return true; - } - unpartitionedDriversScheduled = true; - return false; - } - } - private class DriverSplitRunnerFactory { private final DriverFactory driverFactory; @@ -942,15 +608,14 @@ private DriverSplitRunnerFactory(DriverFactory driverFactory, boolean partitione // TODO: split this method into two: createPartitionedDriverRunner and createUnpartitionedDriverRunner. // The former will take two arguments, and the latter will take one. This will simplify the signature quite a bit. - public DriverSplitRunner createDriverRunner(@Nullable ScheduledSplit partitionedSplit, Lifespan lifespan) + public DriverSplitRunner createDriverRunner(@Nullable ScheduledSplit partitionedSplit) { - checkLifespan(driverFactory.getPipelineExecutionStrategy(), lifespan); - status.incrementPendingCreation(pipelineContext.getPipelineId(), lifespan); + status.incrementPendingCreation(pipelineContext.getPipelineId()); // create driver context immediately so the driver existence is recorded in the stats // the number of drivers is used to balance work across nodes long splitWeight = partitionedSplit == null ? 0 : partitionedSplit.getSplit().getSplitWeight().getRawValue(); - DriverContext driverContext = pipelineContext.addDriverContext(lifespan, splitWeight); - return new DriverSplitRunner(this, driverContext, partitionedSplit, lifespan); + DriverContext driverContext = pipelineContext.addDriverContext(splitWeight); + return new DriverSplitRunner(this, driverContext, partitionedSplit); } public Driver createDriver(DriverContext driverContext, @Nullable ScheduledSplit partitionedSplit) @@ -975,17 +640,15 @@ public Driver createDriver(DriverContext driverContext, @Nullable ScheduledSplit } } - status.decrementPendingCreation(pipelineContext.getPipelineId(), driverContext.getLifespan()); + status.decrementPendingCreation(pipelineContext.getPipelineId()); closeDriverFactoryIfFullyCreated(); return driver; } - public void noMoreDriverRunner(Iterable lifespans) + public void noMoreDriverRunner() { - for (Lifespan lifespan : lifespans) { - status.setNoMoreDriverRunner(pipelineContext.getPipelineId(), lifespan); - } + status.setNoMoreDriverRunner(pipelineContext.getPipelineId()); closeDriverFactoryIfFullyCreated(); } @@ -999,9 +662,6 @@ public void closeDriverFactoryIfFullyCreated() if (closed) { return; } - for (Lifespan lifespan : status.getAndAcknowledgeLifespansWithNoMoreDrivers(pipelineContext.getPipelineId())) { - driverFactory.noMoreDrivers(lifespan); - } if (!isNoMoreDriverRunner() || status.getPendingCreation(pipelineContext.getPipelineId()) != 0) { return; } @@ -1009,11 +669,6 @@ public void closeDriverFactoryIfFullyCreated() closed = true; } - public PipelineExecutionStrategy getPipelineExecutionStrategy() - { - return driverFactory.getPipelineExecutionStrategy(); - } - public OptionalInt getDriverInstances() { return driverFactory.getDriverInstances(); @@ -1030,7 +685,6 @@ private static class DriverSplitRunner { private final DriverSplitRunnerFactory driverSplitRunnerFactory; private final DriverContext driverContext; - private final Lifespan lifespan; @GuardedBy("this") private boolean closed; @@ -1041,12 +695,11 @@ private static class DriverSplitRunner @GuardedBy("this") private Driver driver; - private DriverSplitRunner(DriverSplitRunnerFactory driverSplitRunnerFactory, DriverContext driverContext, @Nullable ScheduledSplit partitionedSplit, Lifespan lifespan) + private DriverSplitRunner(DriverSplitRunnerFactory driverSplitRunnerFactory, DriverContext driverContext, @Nullable ScheduledSplit partitionedSplit) { this.driverSplitRunnerFactory = requireNonNull(driverSplitRunnerFactory, "driverSplitRunnerFactory is null"); this.driverContext = requireNonNull(driverContext, "driverContext is null"); this.partitionedSplit = partitionedSplit; - this.lifespan = requireNonNull(lifespan, "lifespan is null"); } public synchronized DriverContext getDriverContext() @@ -1057,11 +710,6 @@ public synchronized DriverContext getDriverContext() return driver.getDriverContext(); } - public Lifespan getLifespan() - { - return lifespan; - } - @Override public synchronized boolean isFinished() { @@ -1143,116 +791,59 @@ private static class Status // pending creation: number of created DriverSplitRunners that haven't created underlying Driver. // remaining driver: number of created Drivers that haven't yet finished. - private final TaskContext taskContext; - @GuardedBy("this") private final int pipelineWithTaskLifeCycleCount; - @GuardedBy("this") - private final int pipelineWithDriverGroupLifeCycleCount; // For these 3 perX fields, they are populated lazily. If enumeration operations on the // map can lead to side effects, no new entries can be created after such enumeration has // happened. Otherwise, the order of entry creation and the enumeration operation will // lead to different outcome. @GuardedBy("this") - private final Map> perPipelineAndLifespan; - @GuardedBy("this") private final Map perPipeline; @GuardedBy("this") - private final Map perLifespan = new HashMap<>(); + int pipelinesWithNoMoreDriverRunners; @GuardedBy("this") private int overallRemainingDriver; - @GuardedBy("this") - private boolean noMoreLifespans; - - public Status(TaskContext taskContext, Map pipelineToExecutionStrategy) + public Status(Set pipelineIds) { - this.taskContext = requireNonNull(taskContext, "taskContext is null"); int pipelineWithTaskLifeCycleCount = 0; - int pipelineWithDriverGroupLifeCycleCount = 0; - ImmutableMap.Builder> perPipelineAndLifespan = ImmutableMap.builder(); ImmutableMap.Builder perPipeline = ImmutableMap.builder(); - for (Entry entry : pipelineToExecutionStrategy.entrySet()) { - int pipelineId = entry.getKey(); - PipelineExecutionStrategy executionStrategy = entry.getValue(); - perPipelineAndLifespan.put(pipelineId, new HashMap<>()); - perPipeline.put(pipelineId, new PerPipelineStatus(executionStrategy)); - switch (executionStrategy) { - case UNGROUPED_EXECUTION: - pipelineWithTaskLifeCycleCount++; - break; - case GROUPED_EXECUTION: - pipelineWithDriverGroupLifeCycleCount++; - break; - default: - throw new IllegalArgumentException(format("Unknown ExecutionStrategy (%s) for pipeline %s.", executionStrategy, pipelineId)); - } + for (int pipelineId : pipelineIds) { + perPipeline.put(pipelineId, new PerPipelineStatus()); + pipelineWithTaskLifeCycleCount++; } this.pipelineWithTaskLifeCycleCount = pipelineWithTaskLifeCycleCount; - this.pipelineWithDriverGroupLifeCycleCount = pipelineWithDriverGroupLifeCycleCount; - this.perPipelineAndLifespan = perPipelineAndLifespan.buildOrThrow(); this.perPipeline = perPipeline.buildOrThrow(); } - public synchronized void setNoMoreLifespans() - { - if (noMoreLifespans) { - return; - } - noMoreLifespans = true; - } - - public synchronized void setNoMoreDriverRunner(int pipelineId, Lifespan lifespan) + public synchronized void setNoMoreDriverRunner(int pipelineId) { - if (per(pipelineId, lifespan).noMoreDriverRunner) { - return; - } - per(pipelineId, lifespan).noMoreDriverRunner = true; - if (per(pipelineId, lifespan).pendingCreation == 0) { - per(pipelineId).unacknowledgedLifespansWithNoMoreDrivers.add(lifespan); - } - per(pipelineId).lifespansWithNoMoreDriverRunners++; - per(lifespan).pipelinesWithNoMoreDriverRunners++; - checkLifespanCompletion(lifespan); + per(pipelineId).noMoreDriverRunners = true; + pipelinesWithNoMoreDriverRunners++; } - public synchronized void incrementPendingCreation(int pipelineId, Lifespan lifespan) + public synchronized void incrementPendingCreation(int pipelineId) { - checkState(!per(pipelineId, lifespan).noMoreDriverRunner, "Cannot increment pendingCreation for Pipeline %s Lifespan %s. NoMoreSplits is set.", pipelineId, lifespan); - per(pipelineId, lifespan).pendingCreation++; per(pipelineId).pendingCreation++; } - public synchronized void decrementPendingCreation(int pipelineId, Lifespan lifespan) + public synchronized void decrementPendingCreation(int pipelineId) { - checkState(per(pipelineId, lifespan).pendingCreation > 0, "Cannot decrement pendingCreation for Pipeline %s Lifespan %s. Value is 0.", pipelineId, lifespan); - per(pipelineId, lifespan).pendingCreation--; - if (per(pipelineId, lifespan).pendingCreation == 0 && per(pipelineId, lifespan).noMoreDriverRunner) { - per(pipelineId).unacknowledgedLifespansWithNoMoreDrivers.add(lifespan); - } per(pipelineId).pendingCreation--; } - public synchronized void incrementRemainingDriver(Lifespan lifespan) + public synchronized void incrementRemainingDriver() { - checkState(!isNoMoreDriverRunners(lifespan), "Cannot increment remainingDriver for Lifespan %s. NoMoreSplits is set.", lifespan); - per(lifespan).remainingDriver++; + checkState(!(pipelinesWithNoMoreDriverRunners == pipelineWithTaskLifeCycleCount), "Cannot increment remainingDriver. NoMoreSplits is set."); overallRemainingDriver++; } - public synchronized void decrementRemainingDriver(Lifespan lifespan) + public synchronized void decrementRemainingDriver() { - checkState(per(lifespan).remainingDriver > 0, "Cannot decrement remainingDriver for Lifespan %s. Value is 0.", lifespan); - per(lifespan).remainingDriver--; + checkState(overallRemainingDriver > 0, "Cannot decrement remainingDriver. Value is 0."); overallRemainingDriver--; - checkLifespanCompletion(lifespan); - } - - public synchronized boolean isNoMoreLifespans() - { - return noMoreLifespans; } public synchronized int getPendingCreation(int pipelineId) @@ -1260,11 +851,6 @@ public synchronized int getPendingCreation(int pipelineId) return per(pipelineId).pendingCreation; } - public synchronized int getRemainingDriver(Lifespan lifespan) - { - return per(lifespan).remainingDriver; - } - public synchronized int getRemainingDriver() { return overallRemainingDriver; @@ -1272,75 +858,7 @@ public synchronized int getRemainingDriver() public synchronized boolean isNoMoreDriverRunners(int pipelineId) { - int driverGroupCount; - switch (per(pipelineId).executionStrategy) { - case UNGROUPED_EXECUTION: - // Even if noMoreLifespans is not set, UNGROUPED_EXECUTION pipelines can only have 1 driver group by nature. - driverGroupCount = 1; - break; - case GROUPED_EXECUTION: - if (!noMoreLifespans) { - // There may still be new driver groups, which means potentially new splits. - return false; - } - - // We are trying to figure out the number of driver life cycles that has this pipeline here. - // Since the pipeline has grouped execution strategy, all Lifespans except for the task-wide one - // should have this pipeline. - // Therefore, we get the total number of Lifespans in the task, and deduct 1 if the task-wide one exists. - driverGroupCount = perLifespan.size(); - if (perLifespan.containsKey(Lifespan.taskWide())) { - driverGroupCount--; - } - break; - default: - throw new UnsupportedOperationException(); - } - return per(pipelineId).lifespansWithNoMoreDriverRunners == driverGroupCount; - } - - public synchronized boolean isNoMoreDriverRunners(Lifespan lifespan) - { - if (!lifespan.isTaskWide()) { - return per(lifespan).pipelinesWithNoMoreDriverRunners == pipelineWithDriverGroupLifeCycleCount; - } - else { - return per(lifespan).pipelinesWithNoMoreDriverRunners == pipelineWithTaskLifeCycleCount; - } - } - - /** - * Return driver groups who recently became known to not need any new drivers. - * Once it is determined that a driver group will not need any new driver groups, - * the driver group will be returned in the next invocation of this method. - * Once a driver group is returned, it is considered acknowledged, and will not be returned again. - * In other words, each driver group will be returned by this method only once. - */ - public synchronized List getAndAcknowledgeLifespansWithNoMoreDrivers(int pipelineId) - { - List result = ImmutableList.copyOf(per(pipelineId).unacknowledgedLifespansWithNoMoreDrivers); - per(pipelineId).unacknowledgedLifespansWithNoMoreDrivers.clear(); - return result; - } - - private void checkLifespanCompletion(Lifespan lifespan) - { - if (lifespan.isTaskWide()) { - return; // not a driver group - } - if (!isNoMoreDriverRunners(lifespan)) { - return; - } - if (getRemainingDriver(lifespan) != 0) { - return; - } - taskContext.addCompletedDriverGroup(lifespan); - } - - @GuardedBy("this") - private PerPipelineAndLifespanStatus per(int pipelineId, Lifespan lifespan) - { - return perPipelineAndLifespan.get(pipelineId).computeIfAbsent(lifespan, ignored -> new PerPipelineAndLifespanStatus()); + return per(pipelineId).noMoreDriverRunners; } @GuardedBy("this") @@ -1348,42 +866,11 @@ private PerPipelineStatus per(int pipelineId) { return perPipeline.get(pipelineId); } - - @GuardedBy("this") - private PerLifespanStatus per(Lifespan lifespan) - { - if (perLifespan.containsKey(lifespan)) { - return perLifespan.get(lifespan); - } - PerLifespanStatus result = new PerLifespanStatus(); - perLifespan.put(lifespan, result); - return result; - } } private static class PerPipelineStatus - { - final PipelineExecutionStrategy executionStrategy; - - int pendingCreation; - int lifespansWithNoMoreDriverRunners; - final List unacknowledgedLifespansWithNoMoreDrivers = new ArrayList<>(); - - public PerPipelineStatus(PipelineExecutionStrategy executionStrategy) - { - this.executionStrategy = requireNonNull(executionStrategy, "executionStrategy is null"); - } - } - - private static class PerLifespanStatus - { - int remainingDriver; - int pipelinesWithNoMoreDriverRunners; - } - - private static class PerPipelineAndLifespanStatus { int pendingCreation; - boolean noMoreDriverRunner; + boolean noMoreDriverRunners; } } diff --git a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java index 69cdd203e15b..735f9319ba7e 100644 --- a/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/SqlTaskExecutionFactory.java @@ -81,7 +81,6 @@ public SqlTaskExecution create( fragment.getRoot(), TypeProvider.copyOf(fragment.getSymbols()), fragment.getPartitioningScheme(), - fragment.getStageExecutionDescriptor(), fragment.getPartitionedSources(), outputBuffer); } diff --git a/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java b/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java index c160b533a504..b7e035c1d102 100644 --- a/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java +++ b/core/trino-main/src/main/java/io/trino/execution/TaskStatus.java @@ -16,13 +16,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; import io.airlift.units.DataSize; import io.airlift.units.Duration; import java.net.URI; import java.util.List; -import java.util.Set; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; @@ -52,7 +50,6 @@ public class TaskStatus private final TaskState state; private final URI self; private final String nodeId; - private final Set completedDriverGroups; private final int queuedPartitionedDrivers; private final long queuedPartitionedSplitsWeight; @@ -79,7 +76,6 @@ public TaskStatus( @JsonProperty("state") TaskState state, @JsonProperty("self") URI self, @JsonProperty("nodeId") String nodeId, - @JsonProperty("completedDriverGroups") Set completedDriverGroups, @JsonProperty("failures") List failures, @JsonProperty("queuedPartitionedDrivers") int queuedPartitionedDrivers, @JsonProperty("runningPartitionedDrivers") int runningPartitionedDrivers, @@ -102,7 +98,6 @@ public TaskStatus( this.state = requireNonNull(state, "state is null"); this.self = requireNonNull(self, "self is null"); this.nodeId = requireNonNull(nodeId, "nodeId is null"); - this.completedDriverGroups = requireNonNull(completedDriverGroups, "completedDriverGroups is null"); checkArgument(queuedPartitionedDrivers >= 0, "queuedPartitionedDrivers must be positive"); this.queuedPartitionedDrivers = queuedPartitionedDrivers; @@ -166,12 +161,6 @@ public String getNodeId() return nodeId; } - @JsonProperty - public Set getCompletedDriverGroups() - { - return completedDriverGroups; - } - @JsonProperty public List getFailures() { @@ -268,7 +257,6 @@ public static TaskStatus initialTaskStatus(TaskId taskId, URI location, String n PLANNED, location, nodeId, - ImmutableSet.of(), ImmutableList.of(), 0, 0, @@ -293,7 +281,6 @@ public static TaskStatus failWith(TaskStatus taskStatus, TaskState state, List Lifespan.taskWide())), allSourcePlanNodeIds, Optional.of(memoryRequirements.getRequiredMemory())).orElseThrow(() -> new VerifyException("stage execution is expected to be active")); @@ -574,7 +567,7 @@ private static Multimap createRemoteSplits(Multimap result = ImmutableListMultimap.builder(); for (PlanNodeId planNodeId : exchangeSourceHandles.keySet()) { - result.put(planNodeId, new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new SpoolingExchangeInput(ImmutableList.copyOf(exchangeSourceHandles.get(planNodeId)))), Lifespan.taskWide())); + result.put(planNodeId, new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new SpoolingExchangeInput(ImmutableList.copyOf(exchangeSourceHandles.get(planNodeId)))))); } return result.build(); } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java index cd8d171d4df5..84f6aa3e714a 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedCountScheduler.java @@ -39,7 +39,7 @@ public interface TaskScheduler public FixedCountScheduler(StageExecution stageExecution, List partitionToNode) { requireNonNull(stageExecution, "stage is null"); - this.taskScheduler = (node, partition) -> stageExecution.scheduleTask(node, partition, ImmutableMultimap.of(), ImmutableMultimap.of()); + this.taskScheduler = (node, partition) -> stageExecution.scheduleTask(node, partition, ImmutableMultimap.of()); this.partitionToNode = requireNonNull(partitionToNode, "partitionToNode is null"); } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java index a74941fd1c84..57e0237ff2f6 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/FixedSourcePartitionedScheduler.java @@ -18,18 +18,12 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import io.airlift.log.Logger; -import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.TableExecuteContextManager; import io.trino.execution.scheduler.ScheduleResult.BlockedReason; -import io.trino.execution.scheduler.group.DynamicLifespanScheduler; -import io.trino.execution.scheduler.group.FixedLifespanScheduler; -import io.trino.execution.scheduler.group.LifespanScheduler; import io.trino.metadata.InternalNode; import io.trino.metadata.Split; -import io.trino.operator.StageExecutionDescriptor; import io.trino.server.DynamicFilterService; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.split.SplitSource; import io.trino.sql.planner.plan.PlanNodeId; @@ -39,16 +33,13 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.OptionalInt; import java.util.Set; import java.util.function.Supplier; 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.airlift.concurrent.MoreFutures.whenAnyComplete; import static io.trino.execution.scheduler.SourcePartitionedScheduler.newSourcePartitionedSchedulerAsSourceScheduler; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.Objects.requireNonNull; public class FixedSourcePartitionedScheduler @@ -59,8 +50,6 @@ public class FixedSourcePartitionedScheduler private final StageExecution stageExecution; private final List nodes; private final List sourceSchedulers; - private final List partitionHandles; - private final Optional groupedLifespanScheduler; private final PartitionIdAllocator partitionIdAllocator; private final Map scheduledTasks; @@ -68,14 +57,11 @@ public class FixedSourcePartitionedScheduler public FixedSourcePartitionedScheduler( StageExecution stageExecution, Map splitSources, - StageExecutionDescriptor stageExecutionDescriptor, List schedulingOrder, List nodes, BucketNodeMap bucketNodeMap, int splitBatchSize, - OptionalInt concurrentLifespansPerTask, NodeSelector nodeSelector, - List partitionHandles, DynamicFilterService dynamicFilterService, TableExecuteContextManager tableExecuteContextManager) { @@ -83,38 +69,23 @@ public FixedSourcePartitionedScheduler( requireNonNull(splitSources, "splitSources is null"); requireNonNull(bucketNodeMap, "bucketNodeMap is null"); checkArgument(!requireNonNull(nodes, "nodes is null").isEmpty(), "nodes is empty"); - requireNonNull(partitionHandles, "partitionHandles is null"); requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); this.stageExecution = stageExecution; this.nodes = ImmutableList.copyOf(nodes); - this.partitionHandles = ImmutableList.copyOf(partitionHandles); checkArgument(splitSources.keySet().equals(ImmutableSet.copyOf(schedulingOrder))); BucketedSplitPlacementPolicy splitPlacementPolicy = new BucketedSplitPlacementPolicy(nodeSelector, nodes, bucketNodeMap, stageExecution::getAllTasks); ArrayList sourceSchedulers = new ArrayList<>(); - checkArgument( - partitionHandles.equals(ImmutableList.of(NOT_PARTITIONED)) != stageExecutionDescriptor.isStageGroupedExecution(), - "PartitionHandles should be [NOT_PARTITIONED] if and only if all scan nodes use ungrouped execution strategy"); - int nodeCount = nodes.size(); - int concurrentLifespans; - if (concurrentLifespansPerTask.isPresent() && concurrentLifespansPerTask.getAsInt() * nodeCount <= partitionHandles.size()) { - concurrentLifespans = concurrentLifespansPerTask.getAsInt() * nodeCount; - } - else { - concurrentLifespans = partitionHandles.size(); - } boolean firstPlanNode = true; - Optional groupedLifespanScheduler = Optional.empty(); partitionIdAllocator = new PartitionIdAllocator(); scheduledTasks = new HashMap<>(); for (PlanNodeId planNodeId : schedulingOrder) { SplitSource splitSource = splitSources.get(planNodeId); - boolean groupedExecutionForScanNode = stageExecutionDescriptor.isScanGroupedExecution(planNodeId); // TODO : change anySourceTaskBlocked to accommodate the correct blocked status of source tasks // (ref : https://github.com/trinodb/trino/issues/4713) SourceScheduler sourceScheduler = newSourcePartitionedSchedulerAsSourceScheduler( @@ -122,59 +93,22 @@ public FixedSourcePartitionedScheduler( planNodeId, splitSource, splitPlacementPolicy, - Math.max(splitBatchSize / concurrentLifespans, 1), - groupedExecutionForScanNode, + splitBatchSize, dynamicFilterService, tableExecuteContextManager, () -> true, partitionIdAllocator, scheduledTasks); - if (stageExecutionDescriptor.isStageGroupedExecution() && !groupedExecutionForScanNode) { - sourceScheduler = new AsGroupedSourceScheduler(sourceScheduler); - } sourceSchedulers.add(sourceScheduler); if (firstPlanNode) { firstPlanNode = false; - if (!stageExecutionDescriptor.isStageGroupedExecution()) { - sourceScheduler.startLifespan(Lifespan.taskWide(), NOT_PARTITIONED); - sourceScheduler.noMoreLifespans(); - } - else { - LifespanScheduler lifespanScheduler; - if (bucketNodeMap.isDynamic()) { - // Callee of the constructor guarantees dynamic bucket node map will only be - // used when the stage has no remote source. - // - // When the stage has no remote source, any scan is grouped execution guarantees - // all scan is grouped execution. - lifespanScheduler = new DynamicLifespanScheduler(bucketNodeMap, nodes, partitionHandles, concurrentLifespansPerTask); - } - else { - lifespanScheduler = new FixedLifespanScheduler(bucketNodeMap, partitionHandles, concurrentLifespansPerTask); - } - - // Schedule the first few lifespans - lifespanScheduler.scheduleInitial(sourceScheduler); - // Schedule new lifespans for finished ones - stageExecution.addCompletedDriverGroupsChangedListener(lifespanScheduler::onLifespanFinished); - groupedLifespanScheduler = Optional.of(lifespanScheduler); - } } } - this.groupedLifespanScheduler = groupedLifespanScheduler; this.sourceSchedulers = sourceSchedulers; } - private ConnectorPartitionHandle partitionHandleFor(Lifespan lifespan) - { - if (lifespan.isTaskWide()) { - return NOT_PARTITIONED; - } - return partitionHandles.get(lifespan.getId()); - } - @Override public ScheduleResult schedule() { @@ -183,7 +117,7 @@ public ScheduleResult schedule() if (scheduledTasks.isEmpty()) { ImmutableList.Builder newTasksBuilder = ImmutableList.builder(); for (InternalNode node : nodes) { - Optional task = stageExecution.scheduleTask(node, partitionIdAllocator.getNextId(), ImmutableMultimap.of(), ImmutableMultimap.of()); + Optional task = stageExecution.scheduleTask(node, partitionIdAllocator.getNextId(), ImmutableMultimap.of()); if (task.isPresent()) { scheduledTasks.put(node, task.get()); newTasksBuilder.add(task.get()); @@ -196,29 +130,11 @@ public ScheduleResult schedule() List> blocked = new ArrayList<>(); BlockedReason blockedReason = BlockedReason.NO_ACTIVE_DRIVER_GROUP; - if (groupedLifespanScheduler.isPresent()) { - // Start new driver groups on the first scheduler if necessary, - // i.e. when previous ones have finished execution (not finished scheduling). - // - // Invoke schedule method to get a new SettableFuture every time. - // Reusing previously returned SettableFuture could lead to the ListenableFuture retaining too many listeners. - blocked.add(groupedLifespanScheduler.get().schedule(sourceSchedulers.get(0))); - } - int splitsScheduled = 0; Iterator schedulerIterator = sourceSchedulers.iterator(); - List driverGroupsToStart = ImmutableList.of(); - boolean shouldInvokeNoMoreDriverGroups = false; while (schedulerIterator.hasNext()) { SourceScheduler sourceScheduler = schedulerIterator.next(); - for (Lifespan lifespan : driverGroupsToStart) { - sourceScheduler.startLifespan(lifespan, partitionHandleFor(lifespan)); - } - if (shouldInvokeNoMoreDriverGroups) { - sourceScheduler.noMoreLifespans(); - } - ScheduleResult schedule = sourceScheduler.schedule(); splitsScheduled += schedule.getSplitsScheduled(); if (schedule.getBlockedReason().isPresent()) { @@ -230,16 +146,10 @@ public ScheduleResult schedule() allBlocked = false; } - driverGroupsToStart = sourceScheduler.drainCompletedLifespans(); - if (schedule.isFinished()) { stageExecution.schedulingComplete(sourceScheduler.getPlanNodeId()); schedulerIterator.remove(); sourceScheduler.close(); - shouldInvokeNoMoreDriverGroups = true; - } - else { - shouldInvokeNoMoreDriverGroups = false; } } @@ -301,83 +211,5 @@ public List allNodes() { return allNodes; } - - public InternalNode getNodeForBucket(int bucketId) - { - return bucketNodeMap.getAssignedNode(bucketId).get(); - } - } - - private static class AsGroupedSourceScheduler - implements SourceScheduler - { - private final SourceScheduler sourceScheduler; - private boolean started; - private boolean completed; - private final List pendingCompleted; - - public AsGroupedSourceScheduler(SourceScheduler sourceScheduler) - { - this.sourceScheduler = requireNonNull(sourceScheduler, "sourceScheduler is null"); - pendingCompleted = new ArrayList<>(); - } - - @Override - public void start() - { - sourceScheduler.start(); - } - - @Override - public ScheduleResult schedule() - { - return sourceScheduler.schedule(); - } - - @Override - public void close() - { - sourceScheduler.close(); - } - - @Override - public PlanNodeId getPlanNodeId() - { - return sourceScheduler.getPlanNodeId(); - } - - @Override - public void startLifespan(Lifespan lifespan, ConnectorPartitionHandle partitionHandle) - { - pendingCompleted.add(lifespan); - if (started) { - return; - } - started = true; - sourceScheduler.startLifespan(Lifespan.taskWide(), NOT_PARTITIONED); - sourceScheduler.noMoreLifespans(); - } - - @Override - public void noMoreLifespans() - { - checkState(started); - } - - @Override - public List drainCompletedLifespans() - { - if (!completed) { - List lifespans = sourceScheduler.drainCompletedLifespans(); - if (lifespans.isEmpty()) { - return ImmutableList.of(); - } - checkState(ImmutableList.of(Lifespan.taskWide()).equals(lifespans)); - completed = true; - } - List result = ImmutableList.copyOf(pendingCompleted); - pendingCompleted.clear(); - return result; - } } } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java index b51c6e296672..cfcf8dcd370e 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/PipelinedStageExecution.java @@ -19,10 +19,8 @@ import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; -import com.google.common.collect.Sets; import io.airlift.log.Logger; import io.trino.execution.ExecutionFailureInfo; -import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.SqlStage; import io.trino.execution.StageId; @@ -49,7 +47,6 @@ import javax.annotation.concurrent.GuardedBy; import java.net.URI; -import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -62,7 +59,6 @@ import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; @@ -112,7 +108,6 @@ public class PipelinedStageExecution private final Map outputBufferManagers; private final TaskLifecycleListener taskLifecycleListener; private final FailureDetector failureDetector; - private final Executor executor; private final Optional bucketToPartition; private final Map exchangeSources; private final int attempt; @@ -135,10 +130,6 @@ public class PipelinedStageExecution @GuardedBy("this") private final Set completeSources = new HashSet<>(); - // lifespan tracking - private final Set completedDriverGroups = new HashSet<>(); - private final ListenerManager> completedLifespansChangeListeners = new ListenerManager<>(); - public static PipelinedStageExecution createPipelinedStageExecution( SqlStage stage, Map outputBufferManagers, @@ -161,7 +152,6 @@ public static PipelinedStageExecution createPipelinedStageExecution( outputBufferManagers, taskLifecycleListener, failureDetector, - executor, bucketToPartition, exchangeSources.buildOrThrow(), attempt); @@ -175,7 +165,6 @@ private PipelinedStageExecution( Map outputBufferManagers, TaskLifecycleListener taskLifecycleListener, FailureDetector failureDetector, - Executor executor, Optional bucketToPartition, Map exchangeSources, int attempt) @@ -185,7 +174,6 @@ private PipelinedStageExecution( this.outputBufferManagers = ImmutableMap.copyOf(requireNonNull(outputBufferManagers, "outputBufferManagers is null")); this.taskLifecycleListener = requireNonNull(taskLifecycleListener, "taskLifecycleListener is null"); this.failureDetector = requireNonNull(failureDetector, "failureDetector is null"); - this.executor = requireNonNull(executor, "executor is null"); this.bucketToPartition = requireNonNull(bucketToPartition, "bucketToPartition is null"); this.exchangeSources = ImmutableMap.copyOf(requireNonNull(exchangeSources, "exchangeSources is null")); this.attempt = attempt; @@ -217,12 +205,6 @@ public void addStateChangeListener(StateChangeListener stateChangeListene stateMachine.addStateChangeListener(stateChangeListener); } - @Override - public void addCompletedDriverGroupsChangedListener(Consumer> newlyCompletedDriverGroupConsumer) - { - completedLifespansChangeListeners.addListener(newlyCompletedDriverGroupConsumer); - } - @Override public synchronized void beginScheduling() { @@ -310,8 +292,7 @@ public synchronized void failTaskRemotely(TaskId taskId, Throwable failureCause) public synchronized Optional scheduleTask( InternalNode node, int partition, - Multimap initialSplits, - Multimap noMoreSplitsForLifespan) + Multimap initialSplits) { if (stateMachine.getState().isDone()) { return Optional.empty(); @@ -328,7 +309,6 @@ public synchronized Optional scheduleTask( bucketToPartition, outputBuffers, initialSplits, - ImmutableMultimap.of(), ImmutableSet.of(), Optional.empty()); @@ -352,11 +332,9 @@ public synchronized Optional scheduleTask( allTasks.add(task.getTaskId()); task.addSplits(exchangeSplits.build()); - noMoreSplitsForLifespan.forEach(task::noMoreSplits); completeSources.forEach(task::noMoreSplits); task.addStateChangeListener(this::updateTaskStatus); - task.addStateChangeListener(this::updateCompletedDriverGroups); task.start(); @@ -418,23 +396,6 @@ private synchronized void updateTaskStatus(TaskStatus taskStatus) } } - private synchronized void updateCompletedDriverGroups(TaskStatus taskStatus) - { - // Sets.difference returns a view. - // Once we add the difference into `completedDriverGroups`, the view will be empty. - // `completedLifespansChangeListeners.invoke` happens asynchronously. - // As a result, calling the listeners before updating `completedDriverGroups` doesn't make a difference. - // That's why a copy must be made here. - Set newlyCompletedDriverGroups = ImmutableSet.copyOf(Sets.difference(taskStatus.getCompletedDriverGroups(), this.completedDriverGroups)); - if (newlyCompletedDriverGroups.isEmpty()) { - return; - } - completedLifespansChangeListeners.invoke(newlyCompletedDriverGroups, executor); - // newlyCompletedDriverGroups is a view. - // Making changes to completedDriverGroups will change newlyCompletedDriverGroups. - completedDriverGroups.addAll(newlyCompletedDriverGroups); - } - private ExecutionFailureInfo rewriteTransportFailure(ExecutionFailureInfo executionFailureInfo) { if (executionFailureInfo.getRemoteHost() == null || failureDetector.getState(executionFailureInfo.getRemoteHost()) != GONE) { @@ -570,7 +531,7 @@ private static Split createExchangeSplit(RemoteTask sourceTask, RemoteTask desti // Fetch the results from the buffer assigned to the task based on id URI exchangeLocation = sourceTask.getTaskStatus().getSelf(); URI splitLocation = uriBuilderFrom(exchangeLocation).appendPath("results").appendPath(String.valueOf(destinationTask.getTaskId().getPartitionId())).build(); - return new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(sourceTask.getTaskId(), splitLocation.toString())), Lifespan.taskWide()); + return new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(sourceTask.getTaskId(), splitLocation.toString()))); } private static class PipelinedStageStateMachine @@ -666,24 +627,4 @@ public void addStateChangeListener(StateChangeListener stateChangeListene state.addStateChangeListener(stateChangeListener); } } - - private static class ListenerManager - { - private final List> listeners = new ArrayList<>(); - private boolean frozen; - - public synchronized void addListener(Consumer listener) - { - checkState(!frozen, "Listeners have been invoked"); - listeners.add(listener); - } - - public synchronized void invoke(T payload, Executor executor) - { - frozen = true; - for (Consumer listener : listeners) { - executor.execute(() -> listener.accept(payload)); - } - } - } } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java index 2c6055eb0fc1..25dd428bb3b3 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/ScaledWriterScheduler.java @@ -119,7 +119,7 @@ private List scheduleTasks(int count) ImmutableList.Builder tasks = ImmutableList.builder(); for (InternalNode node : nodes) { - Optional remoteTask = stage.scheduleTask(node, scheduledNodes.size(), ImmutableMultimap.of(), ImmutableMultimap.of()); + Optional remoteTask = stage.scheduleTask(node, scheduledNodes.size(), ImmutableMultimap.of()); remoteTask.ifPresent(task -> { tasks.add(task); scheduledNodes.add(node); diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java index 42ab0682953a..e4a6a1e7e3c0 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourcePartitionedScheduler.java @@ -13,22 +13,18 @@ */ package io.trino.execution.scheduler; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; -import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.TableExecuteContext; import io.trino.execution.TableExecuteContextManager; -import io.trino.execution.scheduler.FixedSourcePartitionedScheduler.BucketedSplitPlacementPolicy; import io.trino.metadata.InternalNode; import io.trino.metadata.Split; import io.trino.server.DynamicFilterService; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.split.EmptySplit; import io.trino.split.SplitSource; import io.trino.split.SplitSource.SplitBatch; @@ -37,10 +33,8 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Optional; import java.util.Set; import java.util.concurrent.ThreadLocalRandom; @@ -60,7 +54,6 @@ import static io.trino.execution.scheduler.ScheduleResult.BlockedReason.NO_ACTIVE_DRIVER_GROUP; import static io.trino.execution.scheduler.ScheduleResult.BlockedReason.SPLIT_QUEUES_FULL; import static io.trino.execution.scheduler.ScheduleResult.BlockedReason.WAITING_FOR_SOURCE; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.Objects.requireNonNull; public class SourcePartitionedScheduler @@ -73,20 +66,15 @@ private enum State */ INITIALIZED, - /** - * At least one split has been added to pendingSplits set. - */ - SPLITS_ADDED, - /** * All splits from underlying SplitSource have been discovered. * No more splits will be added to the pendingSplits set. */ - NO_MORE_SPLITS, + SPLITS_SCHEDULED, /** * All splits have been provided to caller of this scheduler. - * Cleanup operations are done (e.g., drainCompletedLifespans has drained all driver groups). + * Cleanup operations are done */ FINISHED } @@ -96,18 +84,18 @@ private enum State private final SplitPlacementPolicy splitPlacementPolicy; private final int splitBatchSize; private final PlanNodeId partitionedNode; - private final boolean groupedExecution; private final DynamicFilterService dynamicFilterService; private final TableExecuteContextManager tableExecuteContextManager; private final BooleanSupplier anySourceTaskBlocked; private final PartitionIdAllocator partitionIdAllocator; private final Map scheduledTasks; - private final Map scheduleGroups = new HashMap<>(); - private boolean noMoreScheduleGroups; + public ListenableFuture nextSplitBatchFuture; + public ListenableFuture placementFuture = immediateVoidFuture(); + public final Set pendingSplits = new HashSet<>(); private State state = State.INITIALIZED; - private SettableFuture whenFinishedOrNewLifespanAdded = SettableFuture.create(); + private SettableFuture whenFinished = SettableFuture.create(); private SourcePartitionedScheduler( StageExecution stageExecution, @@ -115,7 +103,6 @@ private SourcePartitionedScheduler( SplitSource splitSource, SplitPlacementPolicy splitPlacementPolicy, int splitBatchSize, - boolean groupedExecution, DynamicFilterService dynamicFilterService, TableExecuteContextManager tableExecuteContextManager, BooleanSupplier anySourceTaskBlocked, @@ -128,7 +115,6 @@ private SourcePartitionedScheduler( checkArgument(splitBatchSize > 0, "splitBatchSize must be at least one"); this.splitBatchSize = splitBatchSize; this.partitionedNode = requireNonNull(partitionedNode, "partitionedNode is null"); - this.groupedExecution = groupedExecution; this.dynamicFilterService = requireNonNull(dynamicFilterService, "dynamicFilterService is null"); this.tableExecuteContextManager = requireNonNull(tableExecuteContextManager, "tableExecuteContextManager is null"); this.anySourceTaskBlocked = requireNonNull(anySourceTaskBlocked, "anySourceTaskBlocked is null"); @@ -165,14 +151,11 @@ public static StageScheduler newSourcePartitionedSchedulerAsStageScheduler( splitSource, splitPlacementPolicy, splitBatchSize, - false, dynamicFilterService, tableExecuteContextManager, anySourceTaskBlocked, new PartitionIdAllocator(), new HashMap<>()); - sourcePartitionedScheduler.startLifespan(Lifespan.taskWide(), NOT_PARTITIONED); - sourcePartitionedScheduler.noMoreLifespans(); return new StageScheduler() { @@ -185,9 +168,7 @@ public void start() @Override public ScheduleResult schedule() { - ScheduleResult scheduleResult = sourcePartitionedScheduler.schedule(); - sourcePartitionedScheduler.drainCompletedLifespans(); - return scheduleResult; + return sourcePartitionedScheduler.schedule(); } @Override @@ -204,10 +185,6 @@ public void close() * This returns a {@code SourceScheduler} that can be used for a pipeline * that is either ungrouped or grouped. However, the caller is responsible initializing * the driver groups in this scheduler accordingly. - *

- * Besides, the caller is required to poll {@link #drainCompletedLifespans()} - * in addition to {@link #schedule()} on the returned object. Otherwise, lifecycle - * transitioning of the object will not work properly. */ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( StageExecution stageExecution, @@ -215,7 +192,6 @@ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( SplitSource splitSource, SplitPlacementPolicy splitPlacementPolicy, int splitBatchSize, - boolean groupedExecution, DynamicFilterService dynamicFilterService, TableExecuteContextManager tableExecuteContextManager, BooleanSupplier anySourceTaskBlocked, @@ -228,7 +204,6 @@ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( splitSource, splitPlacementPolicy, splitBatchSize, - groupedExecution, dynamicFilterService, tableExecuteContextManager, anySourceTaskBlocked, @@ -236,27 +211,6 @@ public static SourceScheduler newSourcePartitionedSchedulerAsSourceScheduler( scheduledTasks); } - @Override - public synchronized void startLifespan(Lifespan lifespan, ConnectorPartitionHandle partitionHandle) - { - checkState(state == State.INITIALIZED || state == State.SPLITS_ADDED); - scheduleGroups.put(lifespan, new ScheduleGroup(partitionHandle)); - whenFinishedOrNewLifespanAdded.set(null); - whenFinishedOrNewLifespanAdded = SettableFuture.create(); - } - - @Override - public synchronized void noMoreLifespans() - { - checkState(state == State.INITIALIZED || state == State.SPLITS_ADDED); - noMoreScheduleGroups = true; - // The listener is waiting for "new lifespan added" because new lifespans would bring new works to scheduler. - // "No more lifespans" would be of interest to such listeners because it signals that is not going to happen anymore, - // and the listener should stop waiting. - whenFinishedOrNewLifespanAdded.set(null); - whenFinishedOrNewLifespanAdded = SettableFuture.create(); - } - @Override public synchronized void start() { @@ -272,7 +226,7 @@ public synchronized void start() @Override public synchronized ScheduleResult schedule() { - dropListenersFromWhenFinishedOrNewLifespansAdded(); + dropListenersFromWhenFinished(); int overallSplitAssignmentCount = 0; ImmutableSet.Builder overallNewTasks = ImmutableSet.builder(); @@ -281,29 +235,25 @@ public synchronized ScheduleResult schedule() boolean anyBlockedOnNextSplitBatch = false; boolean anyNotBlocked = false; - for (Entry entry : scheduleGroups.entrySet()) { - Lifespan lifespan = entry.getKey(); - ScheduleGroup scheduleGroup = entry.getValue(); - Set pendingSplits = scheduleGroup.pendingSplits; - - if (scheduleGroup.state == ScheduleGroupState.NO_MORE_SPLITS || scheduleGroup.state == ScheduleGroupState.DONE) { - verify(scheduleGroup.nextSplitBatchFuture == null); + if (state != State.FINISHED) { + if (state == State.SPLITS_SCHEDULED) { + verify(nextSplitBatchFuture == null); } else if (pendingSplits.isEmpty()) { // try to get the next batch - if (scheduleGroup.nextSplitBatchFuture == null) { - scheduleGroup.nextSplitBatchFuture = splitSource.getNextBatch(scheduleGroup.partitionHandle, lifespan, splitBatchSize - pendingSplits.size()); + if (nextSplitBatchFuture == null) { + nextSplitBatchFuture = splitSource.getNextBatch(splitBatchSize); long start = System.nanoTime(); - addSuccessCallback(scheduleGroup.nextSplitBatchFuture, () -> stageExecution.recordGetSplitTime(start)); + addSuccessCallback(nextSplitBatchFuture, () -> stageExecution.recordGetSplitTime(start)); } - if (scheduleGroup.nextSplitBatchFuture.isDone()) { - SplitBatch nextSplits = getFutureValue(scheduleGroup.nextSplitBatchFuture); - scheduleGroup.nextSplitBatchFuture = null; + if (nextSplitBatchFuture.isDone()) { + SplitBatch nextSplits = getFutureValue(nextSplitBatchFuture); + nextSplitBatchFuture = null; pendingSplits.addAll(nextSplits.getSplits()); if (nextSplits.isLastBatch()) { - if (scheduleGroup.state == ScheduleGroupState.INITIALIZED && pendingSplits.isEmpty()) { + if (state == State.INITIALIZED && pendingSplits.isEmpty()) { // Add an empty split in case no splits have been produced for the source. // For source operators, they never take input, but they may produce output. // This is well handled by the execution engine. @@ -312,112 +262,77 @@ else if (pendingSplits.isEmpty()) { // Scheduling an empty split kicks off necessary driver instantiation to make this work. pendingSplits.add(new Split( splitSource.getCatalogName(), - new EmptySplit(splitSource.getCatalogName()), - lifespan)); + new EmptySplit(splitSource.getCatalogName()))); } - scheduleGroup.state = ScheduleGroupState.NO_MORE_SPLITS; + state = State.SPLITS_SCHEDULED; } } else { - overallBlockedFutures.add(scheduleGroup.nextSplitBatchFuture); + overallBlockedFutures.add(nextSplitBatchFuture); anyBlockedOnNextSplitBatch = true; - continue; } } - - Multimap splitAssignment = ImmutableMultimap.of(); - if (!pendingSplits.isEmpty()) { - if (!scheduleGroup.placementFuture.isDone()) { - anyBlockedOnPlacements = true; - continue; - } - - if (scheduleGroup.state == ScheduleGroupState.INITIALIZED) { - scheduleGroup.state = ScheduleGroupState.SPLITS_ADDED; - } - if (state == State.INITIALIZED) { - state = State.SPLITS_ADDED; - } - - // calculate placements for splits - SplitPlacementResult splitPlacementResult = splitPlacementPolicy.computeAssignments(pendingSplits); - splitAssignment = splitPlacementResult.getAssignments(); - - // remove splits with successful placements - splitAssignment.values().forEach(pendingSplits::remove); // AbstractSet.removeAll performs terribly here. - overallSplitAssignmentCount += splitAssignment.size(); - - // if not completed placed, mark scheduleGroup as blocked on placement + if (!anyBlockedOnNextSplitBatch) { + Multimap splitAssignment = ImmutableMultimap.of(); + boolean skip = false; if (!pendingSplits.isEmpty()) { - scheduleGroup.placementFuture = splitPlacementResult.getBlocked(); - overallBlockedFutures.add(scheduleGroup.placementFuture); - anyBlockedOnPlacements = true; + if (!placementFuture.isDone()) { + anyBlockedOnPlacements = true; + skip = true; + } + else { // calculate placements for splits + SplitPlacementResult splitPlacementResult = splitPlacementPolicy.computeAssignments(pendingSplits); + splitAssignment = splitPlacementResult.getAssignments(); // remove splits with successful placements + splitAssignment.values().forEach(pendingSplits::remove); // AbstractSet.removeAll performs terribly here. + overallSplitAssignmentCount += splitAssignment.size(); // if not completed placed, mark scheduleGroup as blocked on placement + if (!pendingSplits.isEmpty()) { + placementFuture = splitPlacementResult.getBlocked(); + overallBlockedFutures.add(placementFuture); + anyBlockedOnPlacements = true; + } + } } - } + if (!skip) { // if no new splits will be assigned, update state and attach completion event + if (pendingSplits.isEmpty() && state == State.SPLITS_SCHEDULED) { + state = State.FINISHED; + } - // if no new splits will be assigned, update state and attach completion event - Multimap noMoreSplitsNotification = ImmutableMultimap.of(); - if (pendingSplits.isEmpty() && scheduleGroup.state == ScheduleGroupState.NO_MORE_SPLITS) { - scheduleGroup.state = ScheduleGroupState.DONE; - if (!lifespan.isTaskWide()) { - InternalNode node = ((BucketedSplitPlacementPolicy) splitPlacementPolicy).getNodeForBucket(lifespan.getId()); - noMoreSplitsNotification = ImmutableMultimap.of(node, lifespan); + // assign the splits with successful placements + overallNewTasks.addAll(assignSplits(splitAssignment)); + + // Assert that "placement future is not done" implies "pendingSplits is not empty". + // The other way around is not true. One obvious reason is (un)lucky timing, where the placement is unblocked between `computeAssignments` and this line. + // However, there are other reasons that could lead to this. + // Note that `computeAssignments` is quite broken: + // 1. It always returns a completed future when there are no tasks, regardless of whether all nodes are blocked. + // 2. The returned future will only be completed when a node with an assigned task becomes unblocked. Other nodes don't trigger future completion. + // As a result, to avoid busy loops caused by 1, we check pendingSplits.isEmpty() instead of placementFuture.isDone() here. + if (nextSplitBatchFuture == null && pendingSplits.isEmpty() && state != State.FINISHED) { + anyNotBlocked = true; + } } } - - // assign the splits with successful placements - overallNewTasks.addAll(assignSplits(splitAssignment, noMoreSplitsNotification)); - - // Assert that "placement future is not done" implies "pendingSplits is not empty". - // The other way around is not true. One obvious reason is (un)lucky timing, where the placement is unblocked between `computeAssignments` and this line. - // However, there are other reasons that could lead to this. - // Note that `computeAssignments` is quite broken: - // 1. It always returns a completed future when there are no tasks, regardless of whether all nodes are blocked. - // 2. The returned future will only be completed when a node with an assigned task becomes unblocked. Other nodes don't trigger future completion. - // As a result, to avoid busy loops caused by 1, we check pendingSplits.isEmpty() instead of placementFuture.isDone() here. - if (scheduleGroup.nextSplitBatchFuture == null && scheduleGroup.pendingSplits.isEmpty() && scheduleGroup.state != ScheduleGroupState.DONE) { - anyNotBlocked = true; - } } // * `splitSource.isFinished` invocation may fail after `splitSource.close` has been invoked. - // If state is NO_MORE_SPLITS/FINISHED, splitSource.isFinished has previously returned true, and splitSource is closed now. + // If state is FINISHED, splitSource.isFinished has previously returned true, and splitSource is closed now. // * Even if `splitSource.isFinished()` return true, it is not necessarily safe to tear down the split source. // * If anyBlockedOnNextSplitBatch is true, it means we have not checked out the recently completed nextSplitBatch futures, // which may contain recently published splits. We must not ignore those. - // * If any scheduleGroup is still in DISCOVERING_SPLITS state, it means it hasn't realized that there will be no more splits. - // Next time it invokes getNextBatch, it will realize that. However, the invocation will fail we tear down splitSource now. - if ((state == State.NO_MORE_SPLITS || state == State.FINISHED) || (noMoreScheduleGroups && scheduleGroups.isEmpty() && splitSource.isFinished())) { - switch (state) { - case INITIALIZED: - // We have not scheduled a single split so far. - // But this shouldn't be possible. See usage of EmptySplit in this method. - throw new IllegalStateException("At least 1 split should have been scheduled for this plan node"); - case SPLITS_ADDED: - state = State.NO_MORE_SPLITS; - - Optional> tableExecuteSplitsInfo = splitSource.getTableExecuteSplitsInfo(); - - // Here we assume that we can get non-empty tableExecuteSplitsInfo only for queries which facilitate single split source. - // TODO support grouped execution - tableExecuteSplitsInfo.ifPresent(info -> { - TableExecuteContext tableExecuteContext = tableExecuteContextManager.getTableExecuteContextForQuery(stageExecution.getStageId().getQueryId()); - tableExecuteContext.setSplitsInfo(info); - }); - - splitSource.close(); - // fall through - case NO_MORE_SPLITS: - state = State.FINISHED; - whenFinishedOrNewLifespanAdded.set(null); - // fall through - case FINISHED: - return new ScheduleResult( - true, - overallNewTasks.build(), - overallSplitAssignmentCount); - } - throw new IllegalStateException("Unknown state"); + if (state == State.FINISHED && splitSource.isFinished()) { + Optional> tableExecuteSplitsInfo = splitSource.getTableExecuteSplitsInfo(); + + // Here we assume that we can get non-empty tableExecuteSplitsInfo only for queries which facilitate single split source. + tableExecuteSplitsInfo.ifPresent(info -> { + TableExecuteContext tableExecuteContext = tableExecuteContextManager.getTableExecuteContextForQuery(stageExecution.getStageId().getQueryId()); + tableExecuteContext.setSplitsInfo(info); + }); + + splitSource.close(); + return new ScheduleResult( + true, + overallNewTasks.build(), + overallSplitAssignmentCount); } if (anyNotBlocked) { @@ -431,10 +346,7 @@ else if (pendingSplits.isEmpty()) { dynamicFilterService.unblockStageDynamicFilters(stageExecution.getStageId().getQueryId(), stageExecution.getAttemptId(), stageExecution.getFragment()); } - if (groupedExecution) { - overallNewTasks.addAll(finalizeTaskCreationIfNecessary()); - } - else if (anyBlockedOnPlacements && anySourceTaskBlocked) { + if (anyBlockedOnPlacements && anySourceTaskBlocked) { // In a broadcast join, output buffers of the tasks in build source stage have to // hold onto all data produced before probe side task scheduling finishes, // even if the data is acknowledged by all known consumers. This is because @@ -456,7 +368,15 @@ else if (anyBlockedOnPlacements && anySourceTaskBlocked) { blockedReason = anyBlockedOnPlacements ? SPLIT_QUEUES_FULL : NO_ACTIVE_DRIVER_GROUP; } - overallBlockedFutures.add(whenFinishedOrNewLifespanAdded); + overallBlockedFutures.add(whenFinished); + + if (state == State.FINISHED && splitSource.isFinished()) { + // Wake up blocked caller so that it will invoke schedule() right away. + // Once schedule is invoked, state will be transitioned to FINISHED. + whenFinished.set(null); + whenFinished = SettableFuture.create(); + } + return new ScheduleResult( false, overallNewTasks.build(), @@ -470,23 +390,23 @@ private static ListenableFuture asVoid(ListenableFuture future) return Futures.transform(future, v -> null, directExecutor()); } - private synchronized void dropListenersFromWhenFinishedOrNewLifespansAdded() + private synchronized void dropListenersFromWhenFinished() { - // whenFinishedOrNewLifespanAdded may remain in a not-done state for an extended period of time. + // whenFinished may remain in a not-done state for an extended period of time. // As a result, over time, it can retain a huge number of listener objects. // Whenever schedule is called, holding onto the previous listener is not useful anymore. // Therefore, we drop those listeners here by recreating the future. - // Note: The following implementation is thread-safe because whenFinishedOrNewLifespanAdded can only be completed + // Note: The following implementation is thread-safe because whenFinished can only be completed // while holding the monitor of this. - if (whenFinishedOrNewLifespanAdded.isDone()) { + if (whenFinished.isDone()) { return; } - whenFinishedOrNewLifespanAdded.cancel(true); - whenFinishedOrNewLifespanAdded = SettableFuture.create(); + whenFinished.cancel(true); + whenFinished = SettableFuture.create(); } @Override @@ -495,58 +415,24 @@ public void close() splitSource.close(); } - @Override - public synchronized List drainCompletedLifespans() - { - if (scheduleGroups.isEmpty()) { - // Invoking splitSource.isFinished would fail if it was already closed, which is possible if scheduleGroups is empty. - return ImmutableList.of(); - } - - ImmutableList.Builder result = ImmutableList.builder(); - Iterator> entryIterator = scheduleGroups.entrySet().iterator(); - while (entryIterator.hasNext()) { - Entry entry = entryIterator.next(); - if (entry.getValue().state == ScheduleGroupState.DONE) { - result.add(entry.getKey()); - entryIterator.remove(); - } - } - - if (scheduleGroups.isEmpty() && splitSource.isFinished()) { - // Wake up blocked caller so that it will invoke schedule() right away. - // Once schedule is invoked, state will be transitioned to FINISHED. - whenFinishedOrNewLifespanAdded.set(null); - whenFinishedOrNewLifespanAdded = SettableFuture.create(); - } - - return result.build(); - } - - private Set assignSplits(Multimap splitAssignment, Multimap noMoreSplitsNotification) + private Set assignSplits(Multimap splitAssignment) { ImmutableSet.Builder newTasks = ImmutableSet.builder(); ImmutableSet nodes = ImmutableSet.builder() .addAll(splitAssignment.keySet()) - .addAll(noMoreSplitsNotification.keySet()) .build(); for (InternalNode node : nodes) { // source partitioned tasks can only receive broadcast data; otherwise it would have a different distribution ImmutableMultimap splits = ImmutableMultimap.builder() .putAll(partitionedNode, splitAssignment.get(node)) .build(); - ImmutableMultimap.Builder noMoreSplits = ImmutableMultimap.builder(); - if (noMoreSplitsNotification.containsKey(node)) { - noMoreSplits.putAll(partitionedNode, noMoreSplitsNotification.get(node)); - } RemoteTask task = scheduledTasks.get(node); if (task != null) { task.addSplits(splits); - noMoreSplits.build().forEach(task::noMoreSplits); } else { - scheduleTask(node, splits, noMoreSplits.build()).ifPresent(newTasks::add); + scheduleTask(node, splits).ifPresent(newTasks::add); } } return newTasks.build(); @@ -558,7 +444,7 @@ private void createTaskOnRandomNode() List allNodes = splitPlacementPolicy.allNodes(); checkState(allNodes.size() > 0, "No nodes available"); InternalNode node = allNodes.get(ThreadLocalRandom.current().nextInt(0, allNodes.size())); - scheduleTask(node, ImmutableMultimap.of(), ImmutableMultimap.of()); + scheduleTask(node, ImmutableMultimap.of()); } private Set finalizeTaskCreationIfNecessary() @@ -572,7 +458,7 @@ private Set finalizeTaskCreationIfNecessary() Set newTasks = splitPlacementPolicy.allNodes().stream() .filter(node -> !scheduledTasks.containsKey(node)) - .map(node -> scheduleTask(node, ImmutableMultimap.of(), ImmutableMultimap.of())) + .map(node -> scheduleTask(node, ImmutableMultimap.of())) .filter(Optional::isPresent) .map(Optional::get) .collect(toImmutableSet()); @@ -583,49 +469,10 @@ private Set finalizeTaskCreationIfNecessary() return newTasks; } - private Optional scheduleTask(InternalNode node, Multimap initialSplits, Multimap noMoreSplitsForLifespan) + private Optional scheduleTask(InternalNode node, Multimap initialSplits) { - Optional remoteTask = stageExecution.scheduleTask(node, partitionIdAllocator.getNextId(), initialSplits, noMoreSplitsForLifespan); + Optional remoteTask = stageExecution.scheduleTask(node, partitionIdAllocator.getNextId(), initialSplits); remoteTask.ifPresent(task -> scheduledTasks.put(node, task)); return remoteTask; } - - private static class ScheduleGroup - { - public final ConnectorPartitionHandle partitionHandle; - public ListenableFuture nextSplitBatchFuture; - public ListenableFuture placementFuture = immediateVoidFuture(); - public final Set pendingSplits = new HashSet<>(); - public ScheduleGroupState state = ScheduleGroupState.INITIALIZED; - - public ScheduleGroup(ConnectorPartitionHandle partitionHandle) - { - this.partitionHandle = requireNonNull(partitionHandle, "partitionHandle is null"); - } - } - - private enum ScheduleGroupState - { - /** - * No splits have been added to pendingSplits set. - */ - INITIALIZED, - - /** - * At least one split has been added to pendingSplits set. - */ - SPLITS_ADDED, - - /** - * All splits from underlying SplitSource has been discovered. - * No more splits will be added to the pendingSplits set. - */ - NO_MORE_SPLITS, - - /** - * All splits has been provided to caller of this scheduler. - * Cleanup operations (e.g. inform caller of noMoreSplits) are done. - */ - DONE - } } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourceScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourceScheduler.java index f675eeb59a99..6ae86770523d 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SourceScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SourceScheduler.java @@ -14,12 +14,8 @@ package io.trino.execution.scheduler; -import io.trino.execution.Lifespan; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.sql.planner.plan.PlanNodeId; -import java.util.List; - public interface SourceScheduler { void start(); @@ -29,10 +25,4 @@ public interface SourceScheduler void close(); PlanNodeId getPlanNodeId(); - - void startLifespan(Lifespan lifespan, ConnectorPartitionHandle partitionHandle); - - void noMoreLifespans(); - - List drainCompletedLifespans(); } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java index 7f074118332d..7899e9620b29 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/SqlQueryScheduler.java @@ -61,7 +61,6 @@ import io.trino.spi.ErrorCode; import io.trino.spi.QueryId; import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.exchange.Exchange; import io.trino.spi.exchange.ExchangeContext; import io.trino.spi.exchange.ExchangeId; @@ -122,7 +121,6 @@ import static io.airlift.concurrent.MoreFutures.tryGetFutureValue; import static io.airlift.concurrent.MoreFutures.whenAnyComplete; import static io.airlift.http.client.HttpUriBuilder.uriBuilderFrom; -import static io.trino.SystemSessionProperties.getConcurrentLifespansPerNode; import static io.trino.SystemSessionProperties.getFaultTolerantExecutionPartitionCount; import static io.trino.SystemSessionProperties.getMaxTasksWaitingForNodePerStage; import static io.trino.SystemSessionProperties.getQueryRetryAttempts; @@ -152,7 +150,6 @@ import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.NO_NODES_AVAILABLE; import static io.trino.spi.StandardErrorCode.REMOTE_TASK_FAILED; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION; @@ -1042,7 +1039,6 @@ public void schedule() Optional remoteTask = stageExecution.scheduleTask( coordinator, 0, - ImmutableMultimap.of(), ImmutableMultimap.of()); stageExecution.schedulingComplete(); remoteTask.ifPresent(task -> coordinatorTaskManager.addSourceTaskFailureListener(task.getTaskId(), failureReporter)); @@ -1394,8 +1390,6 @@ public void stateChanged(QueryState newState) NodeSelector nodeSelector = nodeScheduler.createNodeSelector(session, catalogName); SplitPlacementPolicy placementPolicy = new DynamicSplitPlacementPolicy(nodeSelector, stageExecution::getAllTasks); - checkArgument(!fragment.getStageExecutionDescriptor().isStageGroupedExecution()); - return newSourcePartitionedSchedulerAsStageScheduler( stageExecution, planNodeId, @@ -1432,38 +1426,19 @@ else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) { List schedulingOrder = fragment.getPartitionedSources(); Optional catalogName = partitioningHandle.getConnectorId(); checkArgument(catalogName.isPresent(), "No connector ID for partitioning handle: %s", partitioningHandle); - List connectorPartitionHandles; - boolean groupedExecutionForStage = fragment.getStageExecutionDescriptor().isStageGroupedExecution(); - if (groupedExecutionForStage) { - connectorPartitionHandles = nodePartitioningManager.listPartitionHandles(session, partitioningHandle); - checkState(!ImmutableList.of(NOT_PARTITIONED).equals(connectorPartitionHandles)); - } - else { - connectorPartitionHandles = ImmutableList.of(NOT_PARTITIONED); - } BucketNodeMap bucketNodeMap; List stageNodeList; if (fragment.getRemoteSourceNodes().stream().allMatch(node -> node.getExchangeType() == REPLICATE)) { // no remote source - boolean dynamicLifespanSchedule = fragment.getStageExecutionDescriptor().isDynamicLifespanSchedule(); - bucketNodeMap = nodePartitioningManager.getBucketNodeMap(session, partitioningHandle, dynamicLifespanSchedule); - - // verify execution is consistent with planner's decision on dynamic lifespan schedule - verify(bucketNodeMap.isDynamic() == dynamicLifespanSchedule); + bucketNodeMap = nodePartitioningManager.getBucketNodeMap(session, partitioningHandle, false); stageNodeList = new ArrayList<>(nodeScheduler.createNodeSelector(session, catalogName).allNodes()); Collections.shuffle(stageNodeList); } else { - // cannot use dynamic lifespan schedule - verify(!fragment.getStageExecutionDescriptor().isDynamicLifespanSchedule()); - // remote source requires nodePartitionMap NodePartitionMap nodePartitionMap = partitioningCache.apply(partitioningHandle); - if (groupedExecutionForStage) { - checkState(connectorPartitionHandles.size() == nodePartitionMap.getBucketToPartition().length); - } stageNodeList = nodePartitionMap.getPartitionToNode(); bucketNodeMap = nodePartitionMap.asBucketNodeMap(); } @@ -1471,14 +1446,11 @@ else if (partitioningHandle.equals(SCALED_WRITER_DISTRIBUTION)) { return new FixedSourcePartitionedScheduler( stageExecution, splitSources, - fragment.getStageExecutionDescriptor(), schedulingOrder, stageNodeList, bucketNodeMap, splitBatchSize, - getConcurrentLifespansPerNode(session), nodeScheduler.createNodeSelector(session, catalogName), - connectorPartitionHandles, dynamicFilterService, tableExecuteContextManager); } diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java index ca0bfc8d7150..0b64dfa76dba 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageExecution.java @@ -15,7 +15,6 @@ import com.google.common.collect.Multimap; import io.trino.execution.ExecutionFailureInfo; -import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.StageId; import io.trino.execution.StateMachine.StateChangeListener; @@ -28,8 +27,6 @@ import java.util.List; import java.util.Optional; -import java.util.Set; -import java.util.function.Consumer; import static com.google.common.base.Preconditions.checkArgument; @@ -51,8 +48,6 @@ public interface StageExecution void addStateChangeListener(StateChangeListener stateChangeListener); - void addCompletedDriverGroupsChangedListener(Consumer> newlyCompletedDriverGroupConsumer); - TaskLifecycleListener getTaskLifecycleListener(); void schedulingComplete(); @@ -68,8 +63,7 @@ public interface StageExecution Optional scheduleTask( InternalNode node, int partition, - Multimap initialSplits, - Multimap noMoreSplitsForLifespan); + Multimap initialSplits); void failTask(TaskId taskId, Throwable failureCause); diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageTaskSourceFactory.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageTaskSourceFactory.java index a95e001a93f1..1f8f2ab296b9 100644 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/StageTaskSourceFactory.java +++ b/core/trino-main/src/main/java/io/trino/execution/scheduler/StageTaskSourceFactory.java @@ -36,7 +36,6 @@ import io.trino.Session; import io.trino.connector.CatalogName; import io.trino.execution.ForQueryExecution; -import io.trino.execution.Lifespan; import io.trino.execution.QueryManagerConfig; import io.trino.execution.TableExecuteContext; import io.trino.execution.TableExecuteContextManager; @@ -94,7 +93,6 @@ import static io.trino.SystemSessionProperties.getFaultTolerantExecutionTargetTaskSplitCount; import static io.trino.SystemSessionProperties.getFaultTolerantPreserveInputPartitionsInWriteStage; import static io.trino.connector.CatalogName.isInternalSystemConnector; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SCALED_WRITER_DISTRIBUTION; @@ -794,7 +792,7 @@ public synchronized ListenableFuture> getMoreTasks() } checkState(currentSplitBatchFuture.isDone(), "getMoreTasks called again before the previous batch of splits was ready"); - currentSplitBatchFuture = splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), splitBatchSize); + currentSplitBatchFuture = splitSource.getNextBatch(splitBatchSize); long start = System.nanoTime(); addSuccessCallback(currentSplitBatchFuture, () -> getSplitTimeRecorder.accept(start)); @@ -1041,7 +1039,7 @@ public synchronized void load() return; } checkState(currentSplitBatch.isDone(), "next batch of splits requested before previous batch is done"); - currentSplitBatch = splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), splitBatchSize); + currentSplitBatch = splitSource.getNextBatch(splitBatchSize); long start = System.nanoTime(); addCallback( diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/DynamicLifespanScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/group/DynamicLifespanScheduler.java deleted file mode 100644 index b61b1b1274a5..000000000000 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/DynamicLifespanScheduler.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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.scheduler.group; - -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.SettableFuture; -import io.trino.execution.Lifespan; -import io.trino.execution.scheduler.BucketNodeMap; -import io.trino.execution.scheduler.SourceScheduler; -import io.trino.metadata.InternalNode; -import io.trino.spi.connector.ConnectorPartitionHandle; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntListIterator; - -import javax.annotation.concurrent.GuardedBy; - -import java.util.ArrayList; -import java.util.List; -import java.util.OptionalInt; -import java.util.stream.IntStream; - -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 java.util.Collections.unmodifiableList; -import static java.util.Objects.requireNonNull; - -/** - * See {@link LifespanScheduler} about thread safety - */ -public class DynamicLifespanScheduler - implements LifespanScheduler -{ - private final BucketNodeMap bucketNodeMap; - private final List allNodes; - private final List partitionHandles; - private final OptionalInt concurrentLifespansPerTask; - - private final IntListIterator driverGroups; - - // initialScheduled does not need to be guarded because this object - // is safely published after its mutation. - private boolean initialScheduled; - // Write to newDriverGroupReady field is guarded. Read of the reference - // is either guarded, or is guaranteed to happen in the same thread as the write. - private SettableFuture newDriverGroupReady = SettableFuture.create(); - - @GuardedBy("this") - private final List recentlyCompletedDriverGroups = new ArrayList<>(); - - public DynamicLifespanScheduler(BucketNodeMap bucketNodeMap, List allNodes, List partitionHandles, OptionalInt concurrentLifespansPerTask) - { - this.bucketNodeMap = requireNonNull(bucketNodeMap, "bucketNodeMap is null"); - this.allNodes = requireNonNull(allNodes, "allNodes is null"); - this.partitionHandles = unmodifiableList(new ArrayList<>( - requireNonNull(partitionHandles, "partitionHandles is null"))); - - this.concurrentLifespansPerTask = requireNonNull(concurrentLifespansPerTask, "concurrentLifespansPerTask is null"); - concurrentLifespansPerTask.ifPresent(lifespansPerTask -> checkArgument(lifespansPerTask >= 1, "concurrentLifespansPerTask must be great or equal to 1 if present")); - - int bucketCount = partitionHandles.size(); - verify(bucketCount > 0); - this.driverGroups = new IntArrayList(IntStream.range(0, bucketCount).toArray()).iterator(); - } - - @Override - public void scheduleInitial(SourceScheduler scheduler) - { - checkState(!initialScheduled); - initialScheduled = true; - - int driverGroupsScheduledPerTask = 0; - while (driverGroups.hasNext()) { - for (int i = 0; i < allNodes.size() && driverGroups.hasNext(); i++) { - int driverGroupId = driverGroups.nextInt(); - checkState(bucketNodeMap.getAssignedNode(driverGroupId).isEmpty()); - bucketNodeMap.assignBucketToNode(driverGroupId, allNodes.get(i)); - scheduler.startLifespan(Lifespan.driverGroup(driverGroupId), partitionHandles.get(driverGroupId)); - } - - driverGroupsScheduledPerTask++; - if (concurrentLifespansPerTask.isPresent() && driverGroupsScheduledPerTask == concurrentLifespansPerTask.getAsInt()) { - break; - } - } - - if (!driverGroups.hasNext()) { - scheduler.noMoreLifespans(); - } - } - - @Override - public void onLifespanFinished(Iterable newlyCompletedDriverGroups) - { - checkState(initialScheduled); - - SettableFuture newDriverGroupReady; - synchronized (this) { - for (Lifespan newlyCompletedDriverGroup : newlyCompletedDriverGroups) { - checkArgument(!newlyCompletedDriverGroup.isTaskWide()); - recentlyCompletedDriverGroups.add(newlyCompletedDriverGroup); - } - newDriverGroupReady = this.newDriverGroupReady; - } - newDriverGroupReady.set(null); - } - - @Override - public SettableFuture schedule(SourceScheduler scheduler) - { - // Return a new future even if newDriverGroupReady has not finished. - // Returning the same SettableFuture instance could lead to ListenableFuture retaining too many listener objects. - - checkState(initialScheduled); - - List recentlyCompletedDriverGroups; - synchronized (this) { - recentlyCompletedDriverGroups = ImmutableList.copyOf(this.recentlyCompletedDriverGroups); - this.recentlyCompletedDriverGroups.clear(); - newDriverGroupReady = SettableFuture.create(); - } - - for (Lifespan driverGroup : recentlyCompletedDriverGroups) { - if (!driverGroups.hasNext()) { - break; - } - int driverGroupId = driverGroups.nextInt(); - - InternalNode nodeForCompletedDriverGroup = bucketNodeMap.getAssignedNode(driverGroup.getId()).orElseThrow(IllegalStateException::new); - bucketNodeMap.assignBucketToNode(driverGroupId, nodeForCompletedDriverGroup); - scheduler.startLifespan(Lifespan.driverGroup(driverGroupId), partitionHandles.get(driverGroupId)); - } - - if (!driverGroups.hasNext()) { - scheduler.noMoreLifespans(); - } - return newDriverGroupReady; - } -} diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/FixedLifespanScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/group/FixedLifespanScheduler.java deleted file mode 100644 index 2479a69a6590..000000000000 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/FixedLifespanScheduler.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * 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.scheduler.group; - -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.SettableFuture; -import io.trino.execution.Lifespan; -import io.trino.execution.scheduler.BucketNodeMap; -import io.trino.execution.scheduler.SourceScheduler; -import io.trino.metadata.InternalNode; -import io.trino.spi.connector.ConnectorPartitionHandle; -import it.unimi.dsi.fastutil.ints.Int2ObjectMap; -import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap; -import it.unimi.dsi.fastutil.ints.IntArrayList; -import it.unimi.dsi.fastutil.ints.IntList; -import it.unimi.dsi.fastutil.ints.IntListIterator; - -import javax.annotation.concurrent.GuardedBy; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.OptionalInt; - -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 com.google.common.collect.ImmutableMap.toImmutableMap; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; -import static java.util.Objects.requireNonNull; - -/** - * See {@link LifespanScheduler} about thread safety - */ -public class FixedLifespanScheduler - implements LifespanScheduler -{ - private final Int2ObjectMap driverGroupToNodeMap; - private final Map nodeToDriverGroupsMap; - private final List partitionHandles; - private final OptionalInt concurrentLifespansPerTask; - - private boolean initialScheduled; - private SettableFuture newDriverGroupReady = SettableFuture.create(); - @GuardedBy("this") - private final List recentlyCompletedDriverGroups = new ArrayList<>(); - private int totalDriverGroupsScheduled; - - public FixedLifespanScheduler(BucketNodeMap bucketNodeMap, List partitionHandles, OptionalInt concurrentLifespansPerTask) - { - checkArgument(!partitionHandles.equals(ImmutableList.of(NOT_PARTITIONED))); - checkArgument(partitionHandles.size() == bucketNodeMap.getBucketCount()); - - Map nodeToDriverGroupMap = new HashMap<>(); - Int2ObjectMap driverGroupToNodeMap = new Int2ObjectOpenHashMap<>(); - for (int bucket = 0; bucket < bucketNodeMap.getBucketCount(); bucket++) { - InternalNode node = bucketNodeMap.getAssignedNode(bucket).get(); - nodeToDriverGroupMap.computeIfAbsent(node, key -> new IntArrayList()).add(bucket); - driverGroupToNodeMap.put(bucket, node); - } - - this.driverGroupToNodeMap = driverGroupToNodeMap; - this.nodeToDriverGroupsMap = nodeToDriverGroupMap.entrySet().stream() - .collect(toImmutableMap(Map.Entry::getKey, entry -> entry.getValue().iterator())); - this.partitionHandles = requireNonNull(partitionHandles, "partitionHandles is null"); - if (concurrentLifespansPerTask.isPresent()) { - checkArgument(concurrentLifespansPerTask.getAsInt() >= 1, "concurrentLifespansPerTask must be great or equal to 1 if present"); - } - this.concurrentLifespansPerTask = requireNonNull(concurrentLifespansPerTask, "concurrentLifespansPerTask is null"); - } - - @Override - public void scheduleInitial(SourceScheduler scheduler) - { - checkState(!initialScheduled); - initialScheduled = true; - - for (Map.Entry entry : nodeToDriverGroupsMap.entrySet()) { - IntListIterator driverGroupsIterator = entry.getValue(); - int driverGroupsScheduled = 0; - while (driverGroupsIterator.hasNext()) { - int driverGroupId = driverGroupsIterator.nextInt(); - scheduler.startLifespan(Lifespan.driverGroup(driverGroupId), partitionHandles.get(driverGroupId)); - - totalDriverGroupsScheduled++; - driverGroupsScheduled++; - if (concurrentLifespansPerTask.isPresent() && driverGroupsScheduled == concurrentLifespansPerTask.getAsInt()) { - break; - } - } - } - - verify(totalDriverGroupsScheduled <= driverGroupToNodeMap.size()); - if (totalDriverGroupsScheduled == driverGroupToNodeMap.size()) { - scheduler.noMoreLifespans(); - } - } - - @Override - public void onLifespanFinished(Iterable newlyCompletedDriverGroups) - { - checkState(initialScheduled); - - SettableFuture newDriverGroupReady; - synchronized (this) { - for (Lifespan newlyCompletedDriverGroup : newlyCompletedDriverGroups) { - checkArgument(!newlyCompletedDriverGroup.isTaskWide()); - recentlyCompletedDriverGroups.add(newlyCompletedDriverGroup); - } - newDriverGroupReady = this.newDriverGroupReady; - } - newDriverGroupReady.set(null); - } - - @Override - public SettableFuture schedule(SourceScheduler scheduler) - { - // Return a new future even if newDriverGroupReady has not finished. - // Returning the same SettableFuture instance could lead to ListenableFuture retaining too many listener objects. - - checkState(initialScheduled); - - List recentlyCompletedDriverGroups; - synchronized (this) { - recentlyCompletedDriverGroups = ImmutableList.copyOf(this.recentlyCompletedDriverGroups); - this.recentlyCompletedDriverGroups.clear(); - newDriverGroupReady = SettableFuture.create(); - } - - for (Lifespan driverGroup : recentlyCompletedDriverGroups) { - IntListIterator driverGroupsIterator = nodeToDriverGroupsMap.get(driverGroupToNodeMap.get(driverGroup.getId())); - if (!driverGroupsIterator.hasNext()) { - continue; - } - int driverGroupId = driverGroupsIterator.nextInt(); - scheduler.startLifespan(Lifespan.driverGroup(driverGroupId), partitionHandles.get(driverGroupId)); - totalDriverGroupsScheduled++; - } - - verify(totalDriverGroupsScheduled <= driverGroupToNodeMap.size()); - if (totalDriverGroupsScheduled == driverGroupToNodeMap.size()) { - scheduler.noMoreLifespans(); - } - - return newDriverGroupReady; - } -} diff --git a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/LifespanScheduler.java b/core/trino-main/src/main/java/io/trino/execution/scheduler/group/LifespanScheduler.java deleted file mode 100644 index 7511b59b8d8e..000000000000 --- a/core/trino-main/src/main/java/io/trino/execution/scheduler/group/LifespanScheduler.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.scheduler.group; - -import com.google.common.util.concurrent.SettableFuture; -import io.trino.execution.Lifespan; -import io.trino.execution.scheduler.SourceScheduler; - -public interface LifespanScheduler -{ - // Thread Safety: - // * Invocation of onLifespanFinished can be parallel and in any thread. - // There may be multiple invocations in flight at the same time, - // and may overlap with any other methods. - // * Invocation of schedule happens sequentially in a single thread. - // * This object is safely published after invoking scheduleInitial. - - void scheduleInitial(SourceScheduler scheduler); - - void onLifespanFinished(Iterable newlyCompletedDriverGroups); - - SettableFuture schedule(SourceScheduler scheduler); -} diff --git a/core/trino-main/src/main/java/io/trino/metadata/Split.java b/core/trino-main/src/main/java/io/trino/metadata/Split.java index e462afedfec8..c675c8d550be 100644 --- a/core/trino-main/src/main/java/io/trino/metadata/Split.java +++ b/core/trino-main/src/main/java/io/trino/metadata/Split.java @@ -16,7 +16,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; import io.trino.spi.connector.ConnectorSplit; @@ -33,17 +32,14 @@ public final class Split private final CatalogName catalogName; private final ConnectorSplit connectorSplit; - private final Lifespan lifespan; @JsonCreator public Split( @JsonProperty("catalogName") CatalogName catalogName, - @JsonProperty("connectorSplit") ConnectorSplit connectorSplit, - @JsonProperty("lifespan") Lifespan lifespan) + @JsonProperty("connectorSplit") ConnectorSplit connectorSplit) { this.catalogName = requireNonNull(catalogName, "catalogName is null"); this.connectorSplit = requireNonNull(connectorSplit, "connectorSplit is null"); - this.lifespan = requireNonNull(lifespan, "lifespan is null"); } @JsonProperty @@ -58,12 +54,6 @@ public ConnectorSplit getConnectorSplit() return connectorSplit; } - @JsonProperty - public Lifespan getLifespan() - { - return lifespan; - } - public Object getInfo() { return connectorSplit.getInfo(); @@ -90,7 +80,6 @@ public String toString() return toStringHelper(this) .add("catalogName", catalogName) .add("connectorSplit", connectorSplit) - .add("lifespan", lifespan) .toString(); } @@ -98,7 +87,6 @@ public long getRetainedSizeInBytes() { return INSTANCE_SIZE + catalogName.getRetainedSizeInBytes() - + connectorSplit.getRetainedSizeInBytes() - + lifespan.getRetainedSizeInBytes(); + + connectorSplit.getRetainedSizeInBytes(); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/BasicWorkProcessorOperatorAdapter.java b/core/trino-main/src/main/java/io/trino/operator/BasicWorkProcessorOperatorAdapter.java index 665bf1c470c5..90896324566a 100644 --- a/core/trino-main/src/main/java/io/trino/operator/BasicWorkProcessorOperatorAdapter.java +++ b/core/trino-main/src/main/java/io/trino/operator/BasicWorkProcessorOperatorAdapter.java @@ -13,7 +13,6 @@ */ package io.trino.operator; -import io.trino.execution.Lifespan; import io.trino.operator.WorkProcessorOperatorAdapter.AdapterWorkProcessorOperator; import io.trino.operator.WorkProcessorOperatorAdapter.AdapterWorkProcessorOperatorFactory; import io.trino.spi.Page; @@ -92,12 +91,6 @@ public AdapterWorkProcessorOperator createAdapterOperator(ProcessorContext proce return new BasicWorkProcessorOperatorAdapter(processorContext, operatorFactory); } - @Override - public void lifespanFinished(Lifespan lifespan) - { - operatorFactory.lifespanFinished(lifespan); - } - @Override public void close() { diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java index 9ea5d21faa37..9021fdf3231b 100644 --- a/core/trino-main/src/main/java/io/trino/operator/DriverContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/DriverContext.java @@ -20,7 +20,6 @@ import io.airlift.units.DataSize; import io.airlift.units.Duration; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.execution.TaskId; import io.trino.memory.QueryContextVisitor; import io.trino.memory.context.MemoryTrackingContext; @@ -77,7 +76,6 @@ public class DriverContext private final DriverYieldSignal yieldSignal; private final List operatorContexts = new CopyOnWriteArrayList<>(); - private final Lifespan lifespan; private final long splitWeight; public DriverContext( @@ -85,14 +83,12 @@ public DriverContext( Executor notificationExecutor, ScheduledExecutorService yieldExecutor, MemoryTrackingContext driverMemoryContext, - Lifespan lifespan, long splitWeight) { this.pipelineContext = requireNonNull(pipelineContext, "pipelineContext is null"); this.notificationExecutor = requireNonNull(notificationExecutor, "notificationExecutor is null"); this.yieldExecutor = requireNonNull(yieldExecutor, "yieldExecutor is null"); this.driverMemoryContext = requireNonNull(driverMemoryContext, "driverMemoryContext is null"); - this.lifespan = requireNonNull(lifespan, "lifespan is null"); this.yieldSignal = new DriverYieldSignal(); this.splitWeight = splitWeight; checkArgument(splitWeight >= 0, "splitWeight must be >= 0, found: %s", splitWeight); @@ -398,7 +394,6 @@ public DriverStats getDriverStats() } return new DriverStats( - lifespan, createdTime, executionStartTime, executionEndTime, @@ -441,11 +436,6 @@ public List acceptChildren(QueryContextVisitor visitor, C contex .collect(toList()); } - public Lifespan getLifespan() - { - return lifespan; - } - public ScheduledExecutorService getYieldExecutor() { return yieldExecutor; 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 7df8bc810a95..812e5535f325 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 @@ -14,19 +14,14 @@ package io.trino.operator; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Sets; -import io.trino.execution.Lifespan; import io.trino.sql.planner.plan.PlanNodeId; -import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.OptionalInt; -import java.util.Set; 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 com.google.common.collect.ImmutableList.toImmutableList; import static java.util.Objects.requireNonNull; @@ -38,13 +33,10 @@ public class DriverFactory private final List operatorFactories; private final Optional sourceId; private final OptionalInt driverInstances; - private final PipelineExecutionStrategy pipelineExecutionStrategy; private boolean closed; - private final Set encounteredLifespans = new HashSet<>(); - private final Set closedLifespans = new HashSet<>(); - public DriverFactory(int pipelineId, boolean inputDriver, boolean outputDriver, List operatorFactories, OptionalInt driverInstances, PipelineExecutionStrategy pipelineExecutionStrategy) + public DriverFactory(int pipelineId, boolean inputDriver, boolean outputDriver, List operatorFactories, OptionalInt driverInstances) { this.pipelineId = pipelineId; this.inputDriver = inputDriver; @@ -52,7 +44,6 @@ public DriverFactory(int pipelineId, boolean inputDriver, boolean outputDriver, this.operatorFactories = ImmutableList.copyOf(requireNonNull(operatorFactories, "operatorFactories is null")); checkArgument(!operatorFactories.isEmpty(), "There must be at least one operator"); this.driverInstances = requireNonNull(driverInstances, "driverInstances is null"); - this.pipelineExecutionStrategy = requireNonNull(pipelineExecutionStrategy, "pipelineExecutionStrategy is null"); List sourceIds = operatorFactories.stream() .filter(SourceOperatorFactory.class::isInstance) @@ -93,11 +84,6 @@ public OptionalInt getDriverInstances() return driverInstances; } - public PipelineExecutionStrategy getPipelineExecutionStrategy() - { - return pipelineExecutionStrategy; - } - public List getOperatorFactories() { return operatorFactories; @@ -107,8 +93,6 @@ public synchronized Driver createDriver(DriverContext driverContext) { checkState(!closed, "DriverFactory is already closed"); requireNonNull(driverContext, "driverContext is null"); - checkState(!closedLifespans.contains(driverContext.getLifespan()), "DriverFactory is already closed for driver group %s", driverContext.getLifespan()); - encounteredLifespans.add(driverContext.getLifespan()); ImmutableList.Builder operators = ImmutableList.builder(); for (OperatorFactory operatorFactory : operatorFactories) { Operator operator = operatorFactory.createOperator(driverContext); @@ -117,27 +101,11 @@ public synchronized Driver createDriver(DriverContext driverContext) return Driver.createDriver(driverContext, operators.build()); } - public synchronized void noMoreDrivers(Lifespan lifespan) - { - if (closedLifespans.contains(lifespan)) { - return; - } - encounteredLifespans.add(lifespan); - closedLifespans.add(lifespan); - for (OperatorFactory operatorFactory : operatorFactories) { - operatorFactory.noMoreOperators(lifespan); - } - } - public synchronized void noMoreDrivers() { if (closed) { return; } - if (encounteredLifespans.size() != closedLifespans.size()) { - Sets.difference(encounteredLifespans, closedLifespans).forEach(this::noMoreDrivers); - verify(encounteredLifespans.size() == closedLifespans.size()); - } closed = true; for (OperatorFactory operatorFactory : operatorFactories) { operatorFactory.noMoreOperators(); diff --git a/core/trino-main/src/main/java/io/trino/operator/DriverStats.java b/core/trino-main/src/main/java/io/trino/operator/DriverStats.java index 816b3ee4fed1..1fd1954539b8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/DriverStats.java +++ b/core/trino-main/src/main/java/io/trino/operator/DriverStats.java @@ -19,7 +19,6 @@ import com.google.common.collect.ImmutableSet; import io.airlift.units.DataSize; import io.airlift.units.Duration; -import io.trino.execution.Lifespan; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -35,8 +34,6 @@ @Immutable public class DriverStats { - private final Lifespan lifespan; - private final DateTime createTime; private final DateTime startTime; private final DateTime endTime; @@ -80,8 +77,6 @@ public class DriverStats public DriverStats() { - this.lifespan = null; - this.createTime = DateTime.now(); this.startTime = null; this.endTime = null; @@ -125,8 +120,6 @@ public DriverStats() @JsonCreator public DriverStats( - @JsonProperty("lifespan") Lifespan lifespan, - @JsonProperty("createTime") DateTime createTime, @JsonProperty("startTime") DateTime startTime, @JsonProperty("endTime") DateTime endTime, @@ -167,8 +160,6 @@ public DriverStats( @JsonProperty("operatorStats") List operatorStats) { - this.lifespan = lifespan; - this.createTime = requireNonNull(createTime, "createTime is null"); this.startTime = startTime; this.endTime = endTime; @@ -215,12 +206,6 @@ public DriverStats( this.operatorStats = ImmutableList.copyOf(requireNonNull(operatorStats, "operatorStats is null")); } - @JsonProperty - public Lifespan getLifespan() - { - return lifespan; - } - @JsonProperty public DateTime getCreateTime() { diff --git a/core/trino-main/src/main/java/io/trino/operator/OperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/OperatorFactory.java index 6afe12546c0a..5cd54c846575 100644 --- a/core/trino-main/src/main/java/io/trino/operator/OperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/OperatorFactory.java @@ -13,8 +13,6 @@ */ package io.trino.operator; -import io.trino.execution.Lifespan; - public interface OperatorFactory { Operator createOperator(DriverContext driverContext); @@ -25,26 +23,8 @@ public interface OperatorFactory *

* This method will be called only once. * Implementation doesn't need to worry about duplicate invocations. - *

- * It is guaranteed that this will only be invoked after {@link #noMoreOperators(Lifespan)} - * has been invoked for all applicable driver groups. */ void noMoreOperators(); - /** - * Declare that createOperator will not be called any more for the specified Lifespan, - * and release any resources associated with this factory. - *

- * This method will be called only once for each Lifespan. - * Implementation doesn't need to worry about duplicate invocations. - *

- * It is guaranteed that this method will be invoked for all applicable driver groups - * before {@link #noMoreOperators()} is invoked. - */ - default void noMoreOperators(Lifespan lifespan) - { - // do nothing - } - OperatorFactory duplicate(); } diff --git a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java index 011881489bc3..114bfaccbffe 100644 --- a/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/PipelineContext.java @@ -23,7 +23,6 @@ import io.airlift.stats.Distribution; import io.airlift.units.Duration; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.execution.TaskId; import io.trino.memory.QueryContextVisitor; import io.trino.memory.context.LocalMemoryContext; @@ -147,10 +146,10 @@ public boolean isOutputPipeline() public DriverContext addDriverContext() { - return addDriverContext(Lifespan.taskWide(), 0); + return addDriverContext(0); } - public DriverContext addDriverContext(Lifespan lifespan, long splitWeight) + public DriverContext addDriverContext(long splitWeight) { checkArgument(partitioned || splitWeight == 0, "Only partitioned splits should have weights"); DriverContext driverContext = new DriverContext( @@ -158,7 +157,6 @@ public DriverContext addDriverContext(Lifespan lifespan, long splitWeight) notificationExecutor, yieldExecutor, pipelineMemoryContext.newMemoryTrackingContext(), - lifespan, splitWeight); drivers.add(driverContext); return driverContext; diff --git a/core/trino-main/src/main/java/io/trino/operator/PipelineExecutionStrategy.java b/core/trino-main/src/main/java/io/trino/operator/PipelineExecutionStrategy.java deleted file mode 100644 index 65b20a8ed15b..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/PipelineExecutionStrategy.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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.operator; - -public enum PipelineExecutionStrategy -{ - UNGROUPED_EXECUTION, - GROUPED_EXECUTION, -} diff --git a/core/trino-main/src/main/java/io/trino/operator/ProcessorContext.java b/core/trino-main/src/main/java/io/trino/operator/ProcessorContext.java index 70a3af7653c8..64b060512869 100644 --- a/core/trino-main/src/main/java/io/trino/operator/ProcessorContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/ProcessorContext.java @@ -14,7 +14,6 @@ package io.trino.operator; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.execution.TaskId; import io.trino.memory.context.MemoryTrackingContext; @@ -25,7 +24,6 @@ public class ProcessorContext private final Session session; private final MemoryTrackingContext memoryTrackingContext; private final DriverYieldSignal driverYieldSignal; - private final Lifespan lifespan; private final SpillContext spillContext; private final TaskId taskId; @@ -35,7 +33,6 @@ public ProcessorContext(Session session, MemoryTrackingContext memoryTrackingCon this.memoryTrackingContext = requireNonNull(memoryTrackingContext, "memoryTrackingContext is null"); requireNonNull(operatorContext, "operatorContext is null"); this.driverYieldSignal = operatorContext.getDriverContext().getYieldSignal(); - this.lifespan = operatorContext.getDriverContext().getLifespan(); this.spillContext = operatorContext.getSpillContext(); this.taskId = operatorContext.getDriverContext().getTaskId(); } @@ -55,11 +52,6 @@ public DriverYieldSignal getDriverYieldSignal() return driverYieldSignal; } - public Lifespan getLifespan() - { - return lifespan; - } - public SpillContext getSpillContext() { return spillContext; diff --git a/core/trino-main/src/main/java/io/trino/operator/StageExecutionDescriptor.java b/core/trino-main/src/main/java/io/trino/operator/StageExecutionDescriptor.java deleted file mode 100644 index e26d0fcb6c8f..000000000000 --- a/core/trino-main/src/main/java/io/trino/operator/StageExecutionDescriptor.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.operator; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.ImmutableSet; -import io.trino.sql.planner.plan.PlanNodeId; - -import java.util.List; -import java.util.Set; - -import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.operator.StageExecutionDescriptor.StageExecutionStrategy.DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION; -import static io.trino.operator.StageExecutionDescriptor.StageExecutionStrategy.FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION; -import static io.trino.operator.StageExecutionDescriptor.StageExecutionStrategy.UNGROUPED_EXECUTION; -import static java.util.Objects.requireNonNull; - -public class StageExecutionDescriptor -{ - private final StageExecutionStrategy stageExecutionStrategy; - private final Set groupedExecutionScanNodes; - - private StageExecutionDescriptor(StageExecutionStrategy strategy, Set groupedExecutionScanNodes) - { - switch (strategy) { - case UNGROUPED_EXECUTION: - checkArgument(groupedExecutionScanNodes.isEmpty(), "groupedExecutionScanNodes must be empty if stage execution strategy is ungrouped execution"); - break; - case FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION: - case DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION: - checkArgument(!groupedExecutionScanNodes.isEmpty(), "groupedExecutionScanNodes cannot be empty if stage execution strategy is grouped execution"); - break; - default: - throw new IllegalArgumentException("Unsupported stage execution strategy: " + strategy); - } - - this.stageExecutionStrategy = requireNonNull(strategy, "strategy is null"); - this.groupedExecutionScanNodes = requireNonNull(groupedExecutionScanNodes, "groupedExecutionScanNodes is null"); - } - - public static StageExecutionDescriptor ungroupedExecution() - { - return new StageExecutionDescriptor(UNGROUPED_EXECUTION, ImmutableSet.of()); - } - - public static StageExecutionDescriptor fixedLifespanScheduleGroupedExecution(List capableScanNodes) - { - requireNonNull(capableScanNodes, "capableScanNodes is null"); - checkArgument(!capableScanNodes.isEmpty(), "capableScanNodes cannot be empty if stage execution strategy is grouped execution"); - return new StageExecutionDescriptor(FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, ImmutableSet.copyOf(capableScanNodes)); - } - - public static StageExecutionDescriptor dynamicLifespanScheduleGroupedExecution(List capableScanNodes) - { - requireNonNull(capableScanNodes, "capableScanNodes is null"); - checkArgument(!capableScanNodes.isEmpty(), "capableScanNodes cannot be empty if stage execution strategy is grouped execution"); - return new StageExecutionDescriptor(DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, ImmutableSet.copyOf(capableScanNodes)); - } - - @JsonProperty("strategy") - public StageExecutionStrategy getStageExecutionStrategy() - { - return stageExecutionStrategy; - } - - public boolean isStageGroupedExecution() - { - return stageExecutionStrategy != UNGROUPED_EXECUTION; - } - - public boolean isDynamicLifespanSchedule() - { - return stageExecutionStrategy == DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION; - } - - public boolean isScanGroupedExecution(PlanNodeId scanNodeId) - { - return groupedExecutionScanNodes.contains(scanNodeId); - } - - @JsonCreator - public static StageExecutionDescriptor jsonCreator( - @JsonProperty("strategy") StageExecutionStrategy strategy, - @JsonProperty("groupedExecutionScanNodes") Set groupedExecutionCapableScanNodes) - { - return new StageExecutionDescriptor( - requireNonNull(strategy, "strategy is null"), - ImmutableSet.copyOf(requireNonNull(groupedExecutionCapableScanNodes, "groupedExecutionCapableScanNodes is null"))); - } - - @JsonProperty("groupedExecutionScanNodes") - public Set getJsonSerializableGroupedExecutionScanNodes() - { - return groupedExecutionScanNodes; - } - - public enum StageExecutionStrategy - { - UNGROUPED_EXECUTION, - FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION - } -} diff --git a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java index 95e97c4803f6..7aa1618f3768 100644 --- a/core/trino-main/src/main/java/io/trino/operator/TaskContext.java +++ b/core/trino-main/src/main/java/io/trino/operator/TaskContext.java @@ -25,7 +25,6 @@ import io.trino.Session; import io.trino.execution.DynamicFiltersCollector; import io.trino.execution.DynamicFiltersCollector.VersionedDynamicFilterDomains; -import io.trino.execution.Lifespan; import io.trino.execution.TaskId; import io.trino.execution.TaskState; import io.trino.execution.TaskStateMachine; @@ -44,7 +43,6 @@ import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.Executor; import java.util.concurrent.ScheduledExecutorService; @@ -53,7 +51,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Iterables.transform; -import static com.google.common.collect.Sets.newConcurrentHashSet; import static io.airlift.units.DataSize.succinctBytes; import static java.lang.Math.max; import static java.lang.Math.toIntExact; @@ -87,8 +84,6 @@ public class TaskContext private final AtomicReference lastExecutionStartTime = new AtomicReference<>(); private final AtomicReference executionEndTime = new AtomicReference<>(); - private final Set completedDriverGroups = newConcurrentHashSet(); - private final List pipelineContexts = new CopyOnWriteArrayList<>(); private final boolean perOperatorCpuTimerEnabled; @@ -261,22 +256,6 @@ public DataSize getRevocableMemoryReservation() return DataSize.ofBytes(taskMemoryContext.getRevocableMemory()); } - /** - * Returns the completed driver groups (excluding taskWide). - * A driver group is considered complete if all drivers associated with it - * has completed, and no new drivers associated with it will be created. - */ - public Set getCompletedDriverGroups() - { - return completedDriverGroups; - } - - public void addCompletedDriverGroup(Lifespan driverGroup) - { - checkArgument(!driverGroup.isTaskWide(), "driverGroup is task-wide, not a driver group."); - completedDriverGroups.add(driverGroup); - } - public List getPipelineContexts() { return pipelineContexts; diff --git a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorAdapter.java b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorAdapter.java index bc13b4cc693a..e63e4e8668c1 100644 --- a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorAdapter.java +++ b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorAdapter.java @@ -14,7 +14,6 @@ package io.trino.operator; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; import io.trino.memory.context.MemoryTrackingContext; import io.trino.spi.Page; import io.trino.sql.planner.plan.PlanNodeId; @@ -85,12 +84,6 @@ public void noMoreOperators() close(); } - @Override - public void noMoreOperators(Lifespan lifespan) - { - lifespanFinished(lifespan); - } - @Override public OperatorFactory duplicate() { @@ -123,12 +116,6 @@ public WorkProcessorOperator create(ProcessorContext processorContext, WorkProce return operatorFactory.create(processorContext, sourcePages); } - @Override - public void lifespanFinished(Lifespan lifespan) - { - operatorFactory.lifespanFinished(lifespan); - } - @Override public void close() { diff --git a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorFactory.java index 63701a74aacb..f18e6bc8e9be 100644 --- a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorOperatorFactory.java @@ -13,7 +13,6 @@ */ package io.trino.operator; -import io.trino.execution.Lifespan; import io.trino.spi.Page; import io.trino.sql.planner.plan.PlanNodeId; @@ -29,11 +28,6 @@ WorkProcessorOperator create( ProcessorContext processorContext, WorkProcessor sourcePages); - default void lifespanFinished(Lifespan lifespan) - { - //do nothing - } - default void close() { //do nothing diff --git a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorPipelineSourceOperator.java b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorPipelineSourceOperator.java index cb27cbabdf7b..e5cf9afd7a16 100644 --- a/core/trino-main/src/main/java/io/trino/operator/WorkProcessorPipelineSourceOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/WorkProcessorPipelineSourceOperator.java @@ -21,7 +21,6 @@ import io.airlift.log.Logger; import io.airlift.units.DataSize; import io.airlift.units.Duration; -import io.trino.execution.Lifespan; import io.trino.memory.context.AggregatedMemoryContext; import io.trino.memory.context.LocalMemoryContext; import io.trino.memory.context.MemoryTrackingContext; @@ -774,11 +773,5 @@ public void noMoreOperators() this.operatorFactories.forEach(WorkProcessorOperatorFactory::close); closed = true; } - - @Override - public void noMoreOperators(Lifespan lifespan) - { - this.operatorFactories.forEach(operatorFactory -> operatorFactory.lifespanFinished(lifespan)); - } } } diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java index 429bf4f46bb1..5c1173f2499a 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchange.java @@ -19,12 +19,10 @@ import io.airlift.slice.XxHash64; import io.airlift.units.DataSize; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.operator.BucketPartitionFunction; import io.trino.operator.HashGenerator; import io.trino.operator.InterpretedHashGenerator; import io.trino.operator.PartitionFunction; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.PrecomputedHashGenerator; import io.trino.spi.Page; import io.trino.spi.connector.ConnectorBucketNodeMap; @@ -38,23 +36,18 @@ import javax.annotation.concurrent.ThreadSafe; import java.io.Closeable; -import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Stream; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.operator.exchange.LocalExchangeSink.finishedLocalExchangeSink; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_ARBITRARY_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_BROADCAST_DISTRIBUTION; @@ -79,9 +72,6 @@ public class LocalExchange @GuardedBy("this") private boolean noMoreSinkFactories; - @GuardedBy("this") - private final List allSinkFactories; - @GuardedBy("this") private final Set openSinkFactories = new HashSet<>(); @@ -94,22 +84,16 @@ public class LocalExchange public LocalExchange( NodePartitioningManager nodePartitioningManager, Session session, - int sinkFactoryCount, - int bufferCount, + int defaultConcurrency, PartitioningHandle partitioning, List partitionChannels, - List partitionChannelTypes, + List types, Optional partitionHashChannel, DataSize maxBufferedBytes, BlockTypeOperators blockTypeOperators) { - this.allSinkFactories = Stream.generate(() -> new LocalExchangeSinkFactory(LocalExchange.this)) - .limit(sinkFactoryCount) - .collect(toImmutableList()); - openSinkFactories.addAll(allSinkFactories); - noMoreSinkFactories(); - ImmutableList.Builder sources = ImmutableList.builder(); + int bufferCount = computeBufferCount(partitioning, defaultConcurrency, partitionChannels); for (int i = 0; i < bufferCount; i++) { sources.add(new LocalExchangeSource(source -> checkAllSourcesFinished())); } @@ -119,6 +103,10 @@ public LocalExchange( .map(buffer -> (Consumer) buffer::addPage) .collect(toImmutableList()); + List partitionChannelTypes = partitionChannels.stream() + .map(types::get) + .collect(toImmutableList()); + this.memoryManager = new LocalExchangeMemoryManager(maxBufferedBytes.toBytes()); if (partitioning.equals(SINGLE_DISTRIBUTION)) { exchangerSupplier = () -> new BroadcastExchanger(buffers, memoryManager); @@ -185,11 +173,6 @@ public synchronized LocalExchangeSinkFactory createSinkFactory() return newFactory; } - public synchronized LocalExchangeSinkFactory getSinkFactory(LocalExchangeSinkFactoryId id) - { - return allSinkFactories.get(id.id); - } - public synchronized LocalExchangeSource getNextSource() { checkState(nextSourceIndex < sources.size(), "All operators already created"); @@ -341,115 +324,6 @@ private static void checkNotHoldsLock(Object lock) checkState(!Thread.holdsLock(lock), "Cannot execute this method while holding a lock"); } - @ThreadSafe - public static class LocalExchangeFactory - { - private final NodePartitioningManager nodePartitioningManager; - private final Session session; - private final PartitioningHandle partitioning; - private final List partitionChannels; - private final List partitionChannelTypes; - private final Optional partitionHashChannel; - private final PipelineExecutionStrategy exchangeSourcePipelineExecutionStrategy; - private final DataSize maxBufferedBytes; - private final BlockTypeOperators blockTypeOperators; - private final int bufferCount; - - @GuardedBy("this") - private boolean noMoreSinkFactories; - // The number of total sink factories are tracked at planning time - // so that the exact number of sink factory is known by the time execution starts. - @GuardedBy("this") - private int numSinkFactories; - - @GuardedBy("this") - private final Map localExchangeMap = new HashMap<>(); - @GuardedBy("this") - private final List closedSinkFactories = new ArrayList<>(); - - public LocalExchangeFactory( - NodePartitioningManager nodePartitioningManager, - Session session, - PartitioningHandle partitioning, - int defaultConcurrency, - List types, - List partitionChannels, - Optional partitionHashChannel, - PipelineExecutionStrategy exchangeSourcePipelineExecutionStrategy, - DataSize maxBufferedBytes, - BlockTypeOperators blockTypeOperators) - { - this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null"); - this.session = requireNonNull(session, "session is null"); - this.partitioning = requireNonNull(partitioning, "partitioning is null"); - this.partitionChannels = requireNonNull(partitionChannels, "partitionChannels is null"); - requireNonNull(types, "types is null"); - this.partitionChannelTypes = partitionChannels.stream() - .map(types::get) - .collect(toImmutableList()); - this.partitionHashChannel = requireNonNull(partitionHashChannel, "partitionHashChannel is null"); - this.exchangeSourcePipelineExecutionStrategy = requireNonNull(exchangeSourcePipelineExecutionStrategy, "exchangeSourcePipelineExecutionStrategy is null"); - this.maxBufferedBytes = requireNonNull(maxBufferedBytes, "maxBufferedBytes is null"); - this.blockTypeOperators = requireNonNull(blockTypeOperators, "blockTypeOperators is null"); - - this.bufferCount = computeBufferCount(partitioning, defaultConcurrency, partitionChannels); - } - - public synchronized LocalExchangeSinkFactoryId newSinkFactoryId() - { - checkState(!noMoreSinkFactories); - LocalExchangeSinkFactoryId result = new LocalExchangeSinkFactoryId(numSinkFactories); - numSinkFactories++; - return result; - } - - public synchronized void noMoreSinkFactories() - { - noMoreSinkFactories = true; - } - - public int getBufferCount() - { - return bufferCount; - } - - public synchronized LocalExchange getLocalExchange(Lifespan lifespan) - { - if (exchangeSourcePipelineExecutionStrategy == UNGROUPED_EXECUTION) { - checkArgument(lifespan.isTaskWide(), "LocalExchangeFactory is declared as UNGROUPED_EXECUTION. Driver-group exchange cannot be created."); - } - else { - checkArgument(!lifespan.isTaskWide(), "LocalExchangeFactory is declared as GROUPED_EXECUTION. Task-wide exchange cannot be created."); - } - return localExchangeMap.computeIfAbsent(lifespan, ignored -> { - checkState(noMoreSinkFactories); - LocalExchange localExchange = new LocalExchange( - nodePartitioningManager, - session, - numSinkFactories, - bufferCount, - partitioning, - partitionChannels, - partitionChannelTypes, - partitionHashChannel, - maxBufferedBytes, - blockTypeOperators); - for (LocalExchangeSinkFactoryId closedSinkFactoryId : closedSinkFactories) { - localExchange.getSinkFactory(closedSinkFactoryId).close(); - } - return localExchange; - }); - } - - public synchronized void closeSinks(LocalExchangeSinkFactoryId sinkFactoryId) - { - closedSinkFactories.add(sinkFactoryId); - for (LocalExchange localExchange : localExchangeMap.values()) { - localExchange.getSinkFactory(sinkFactoryId).close(); - } - } - } - private static int computeBufferCount(PartitioningHandle partitioning, int defaultConcurrency, List partitionChannels) { int bufferCount; @@ -479,16 +353,6 @@ else if (partitioning.equals(FIXED_HASH_DISTRIBUTION) || partitioning.getConnect return bufferCount; } - public static class LocalExchangeSinkFactoryId - { - private final int id; - - public LocalExchangeSinkFactoryId(int id) - { - this.id = id; - } - } - // Sink factory is entirely a pass thought to LocalExchange. // This class only exists as a separate entity to deal with the complex lifecycle caused // by operator factories (e.g., duplicate and noMoreSinkFactories). diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSinkOperator.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSinkOperator.java index 1a58b4fb5389..b70421a9c7c8 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSinkOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSinkOperator.java @@ -14,15 +14,12 @@ package io.trino.operator.exchange; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; import io.trino.operator.LocalPlannerAware; import io.trino.operator.Operator; import io.trino.operator.OperatorContext; import io.trino.operator.OperatorFactory; -import io.trino.operator.exchange.LocalExchange.LocalExchangeFactory; import io.trino.operator.exchange.LocalExchange.LocalExchangeSinkFactory; -import io.trino.operator.exchange.LocalExchange.LocalExchangeSinkFactoryId; import io.trino.spi.Page; import io.trino.sql.planner.plan.PlanNodeId; @@ -37,23 +34,17 @@ public class LocalExchangeSinkOperator public static class LocalExchangeSinkOperatorFactory implements OperatorFactory, LocalPlannerAware { - private final LocalExchangeFactory localExchangeFactory; - private final int operatorId; - // There will be a LocalExchangeSinkFactory per LocalExchangeSinkOperatorFactory per Driver Group. - // A LocalExchangeSinkOperatorFactory needs to have access to LocalExchangeSinkFactories for each Driver Group. - private final LocalExchangeSinkFactoryId sinkFactoryId; + private final LocalExchangeSinkFactory sinkFactory; private final PlanNodeId planNodeId; private final Function pagePreprocessor; private boolean closed; - public LocalExchangeSinkOperatorFactory(LocalExchangeFactory localExchangeFactory, int operatorId, PlanNodeId planNodeId, LocalExchangeSinkFactoryId sinkFactoryId, Function pagePreprocessor) + public LocalExchangeSinkOperatorFactory(LocalExchangeSinkFactory sinkFactory, int operatorId, PlanNodeId planNodeId, Function pagePreprocessor) { - this.localExchangeFactory = requireNonNull(localExchangeFactory, "localExchangeFactory is null"); - + this.sinkFactory = requireNonNull(sinkFactory, "sinkFactory is null"); this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.sinkFactoryId = requireNonNull(sinkFactoryId, "sinkFactoryId is null"); this.pagePreprocessor = requireNonNull(pagePreprocessor, "pagePreprocessor is null"); } @@ -63,9 +54,7 @@ public Operator createOperator(DriverContext driverContext) checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, LocalExchangeSinkOperator.class.getSimpleName()); - LocalExchangeSinkFactory localExchangeSinkFactory = localExchangeFactory.getLocalExchange(driverContext.getLifespan()).getSinkFactory(sinkFactoryId); - - return new LocalExchangeSinkOperator(operatorContext, localExchangeSinkFactory.createSink(), pagePreprocessor); + return new LocalExchangeSinkOperator(operatorContext, sinkFactory.createSink(), pagePreprocessor); } @Override @@ -73,26 +62,20 @@ public void noMoreOperators() { if (!closed) { closed = true; - localExchangeFactory.closeSinks(sinkFactoryId); + sinkFactory.close(); } } - @Override - public void noMoreOperators(Lifespan lifespan) - { - localExchangeFactory.getLocalExchange(lifespan).getSinkFactory(sinkFactoryId).close(); - } - @Override public OperatorFactory duplicate() { - return new LocalExchangeSinkOperatorFactory(localExchangeFactory, operatorId, planNodeId, localExchangeFactory.newSinkFactoryId(), pagePreprocessor); + return new LocalExchangeSinkOperatorFactory(sinkFactory.duplicate(), operatorId, planNodeId, pagePreprocessor); } @Override public void localPlannerComplete() { - localExchangeFactory.noMoreSinkFactories(); + sinkFactory.noMoreSinkFactories(); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java index 169783db294a..57b6153d3b47 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalExchangeSourceOperator.java @@ -18,7 +18,6 @@ import io.trino.operator.Operator; import io.trino.operator.OperatorContext; import io.trino.operator.OperatorFactory; -import io.trino.operator.exchange.LocalExchange.LocalExchangeFactory; import io.trino.spi.Page; import io.trino.sql.planner.plan.PlanNodeId; @@ -33,14 +32,14 @@ public static class LocalExchangeSourceOperatorFactory { private final int operatorId; private final PlanNodeId planNodeId; - private final LocalExchangeFactory localExchangeFactory; + private final LocalExchange localExchange; private boolean closed; - public LocalExchangeSourceOperatorFactory(int operatorId, PlanNodeId planNodeId, LocalExchangeFactory localExchangeFactory) + public LocalExchangeSourceOperatorFactory(int operatorId, PlanNodeId planNodeId, LocalExchange localExchange) { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.localExchangeFactory = requireNonNull(localExchangeFactory, "localExchangeFactory is null"); + this.localExchange = requireNonNull(localExchange, "localExchange is null"); } @Override @@ -48,10 +47,8 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); - LocalExchange inMemoryExchange = localExchangeFactory.getLocalExchange(driverContext.getLifespan()); - OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, LocalExchangeSourceOperator.class.getSimpleName()); - return new LocalExchangeSourceOperator(operatorContext, inMemoryExchange.getNextSource()); + return new LocalExchangeSourceOperator(operatorContext, localExchange.getNextSource()); } @Override @@ -65,11 +62,6 @@ public OperatorFactory duplicate() { throw new UnsupportedOperationException("Source operator factories cannot be duplicated"); } - - public LocalExchangeFactory getLocalExchangeFactory() - { - return localExchangeFactory; - } } private final OperatorContext operatorContext; diff --git a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java index 815f826c368a..ccf07a621a6f 100644 --- a/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/exchange/LocalMergeSourceOperator.java @@ -21,7 +21,6 @@ import io.trino.operator.OperatorFactory; import io.trino.operator.PageWithPositionComparator; import io.trino.operator.WorkProcessor; -import io.trino.operator.exchange.LocalExchange.LocalExchangeFactory; import io.trino.spi.Page; import io.trino.spi.connector.SortOrder; import io.trino.spi.type.Type; @@ -44,7 +43,7 @@ public static class LocalMergeSourceOperatorFactory { private final int operatorId; private final PlanNodeId planNodeId; - private final LocalExchangeFactory localExchangeFactory; + private final LocalExchange localExchange; private final List types; private final OrderingCompiler orderingCompiler; private final List sortChannels; @@ -54,7 +53,7 @@ public static class LocalMergeSourceOperatorFactory public LocalMergeSourceOperatorFactory( int operatorId, PlanNodeId planNodeId, - LocalExchangeFactory localExchangeFactory, + LocalExchange localExchange, List types, OrderingCompiler orderingCompiler, List sortChannels, @@ -62,7 +61,7 @@ public LocalMergeSourceOperatorFactory( { this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.localExchangeFactory = requireNonNull(localExchangeFactory, "localExchangeFactory is null"); + this.localExchange = requireNonNull(localExchange, "localExchange is null"); this.types = ImmutableList.copyOf(requireNonNull(types, "types is null")); this.orderingCompiler = requireNonNull(orderingCompiler, "orderingCompiler is null"); this.sortChannels = ImmutableList.copyOf(requireNonNull(sortChannels, "sortChannels is null")); @@ -74,8 +73,6 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); - LocalExchange localExchange = localExchangeFactory.getLocalExchange(driverContext.getLifespan()); - OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, LocalMergeSourceOperator.class.getSimpleName()); PageWithPositionComparator comparator = orderingCompiler.compilePageWithPositionComparator(types, sortChannels, orderings); List sources = IntStream.range(0, localExchange.getBufferCount()) diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexBuildDriverFactoryProvider.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexBuildDriverFactoryProvider.java index b2d8e7e68feb..221c7fe27b75 100644 --- a/core/trino-main/src/main/java/io/trino/operator/index/IndexBuildDriverFactoryProvider.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexBuildDriverFactoryProvider.java @@ -25,7 +25,6 @@ import java.util.OptionalInt; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.operator.index.PageBufferOperator.PageBufferOperatorFactory; import static io.trino.operator.index.PagesIndexBuilderOperator.PagesIndexBuilderOperatorFactory; import static java.util.Objects.requireNonNull; @@ -85,8 +84,7 @@ public DriverFactory createSnapshot(int pipelineId, IndexSnapshotBuilder indexSn .addAll(coreOperatorFactories) .add(new PagesIndexBuilderOperatorFactory(outputOperatorId, planNodeId, indexSnapshotBuilder, "IndexBuilder")) .build(), - OptionalInt.empty(), - UNGROUPED_EXECUTION); + OptionalInt.empty()); } public DriverFactory createStreaming(PageBuffer pageBuffer, Page indexKeyTuple) @@ -101,6 +99,6 @@ public DriverFactory createStreaming(PageBuffer pageBuffer, Page indexKeyTuple) operatorFactories.add(new PageBufferOperatorFactory(outputOperatorId, planNodeId, pageBuffer, "IndexBuilder")); - return new DriverFactory(pipelineId, inputDriver, false, operatorFactories.build(), OptionalInt.empty(), UNGROUPED_EXECUTION); + return new DriverFactory(pipelineId, inputDriver, false, operatorFactories.build(), OptionalInt.empty()); } } diff --git a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java index 66ca68882254..339faa82d29d 100644 --- a/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java +++ b/core/trino-main/src/main/java/io/trino/operator/index/IndexLoader.java @@ -18,7 +18,6 @@ import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.DataSize; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.execution.ScheduledSplit; import io.trino.execution.SplitAssignment; import io.trino.metadata.Split; @@ -241,7 +240,7 @@ public IndexedData streamIndexDataForSingleKey(UpdateRequest updateRequest) PageRecordSet pageRecordSet = new PageRecordSet(keyTypes, indexKeyTuple); PlanNodeId planNodeId = driverFactory.getSourceId().get(); - ScheduledSplit split = new ScheduledSplit(0, planNodeId, new Split(INDEX_CONNECTOR_ID, new IndexSplit(pageRecordSet), Lifespan.taskWide())); + ScheduledSplit split = new ScheduledSplit(0, planNodeId, new Split(INDEX_CONNECTOR_ID, new IndexSplit(pageRecordSet))); driver.updateSplitAssignment(new SplitAssignment(planNodeId, ImmutableSet.of(split), true)); return new StreamingIndexedData(outputTypes, keyEqualOperators, indexKeyTuple, pageBuffer, driver); @@ -337,7 +336,7 @@ public boolean load(List requests) // Drive index lookup to produce the output (landing in indexSnapshotBuilder) try (Driver driver = driverFactory.createDriver(pipelineContext.addDriverContext())) { PlanNodeId sourcePlanNodeId = driverFactory.getSourceId().get(); - ScheduledSplit split = new ScheduledSplit(0, sourcePlanNodeId, new Split(INDEX_CONNECTOR_ID, new IndexSplit(recordSetForLookupSource), Lifespan.taskWide())); + ScheduledSplit split = new ScheduledSplit(0, sourcePlanNodeId, new Split(INDEX_CONNECTOR_ID, new IndexSplit(recordSetForLookupSource))); driver.updateSplitAssignment(new SplitAssignment(sourcePlanNodeId, ImmutableSet.of(split), true)); while (!driver.isFinished()) { ListenableFuture process = driver.processUntilBlocked(); diff --git a/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java index d0bbf8cc99da..02119ae65f3a 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/HashBuilderOperator.java @@ -19,7 +19,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.airlift.log.Logger; -import io.trino.execution.Lifespan; import io.trino.memory.context.LocalMemoryContext; import io.trino.operator.DriverContext; import io.trino.operator.HashArraySizeSupplier; @@ -39,9 +38,7 @@ import java.io.IOException; import java.util.ArrayDeque; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.OptionalLong; @@ -83,7 +80,7 @@ public static class HashBuilderOperatorFactory private final SingleStreamSpillerFactory singleStreamSpillerFactory; private final HashArraySizeSupplier hashArraySizeSupplier; - private final Map partitionIndexManager = new HashMap<>(); + private int partitionIndex; private boolean closed; @@ -130,13 +127,13 @@ public HashBuilderOperator createOperator(DriverContext driverContext) checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, HashBuilderOperator.class.getSimpleName()); - PartitionedLookupSourceFactory lookupSourceFactory = this.lookupSourceFactoryManager.getJoinBridge(driverContext.getLifespan()); - int partitionIndex = getAndIncrementPartitionIndex(driverContext.getLifespan()); + PartitionedLookupSourceFactory lookupSourceFactory = this.lookupSourceFactoryManager.getJoinBridge(); verify(partitionIndex < lookupSourceFactory.partitions()); + partitionIndex++; return new HashBuilderOperator( operatorContext, lookupSourceFactory, - partitionIndex, + partitionIndex - 1, outputChannels, hashChannels, preComputedHashChannel, @@ -161,11 +158,6 @@ public OperatorFactory duplicate() { throw new UnsupportedOperationException("Parallel hash build cannot be duplicated"); } - - private int getAndIncrementPartitionIndex(Lifespan lifespan) - { - return partitionIndexManager.compute(lifespan, (k, v) -> v == null ? 1 : v + 1) - 1; - } } @VisibleForTesting diff --git a/core/trino-main/src/main/java/io/trino/operator/join/JoinBridgeManager.java b/core/trino-main/src/main/java/io/trino/operator/join/JoinBridgeManager.java index 44e9cfb910e6..be7f99932f9c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/JoinBridgeManager.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/JoinBridgeManager.java @@ -17,22 +17,16 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.ReferenceCount; import io.trino.spi.type.Type; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.Futures.transform; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static java.util.Objects.requireNonNull; public class JoinBridgeManager @@ -42,34 +36,26 @@ public static JoinBridgeManager lookupAllAtOnce( { return new JoinBridgeManager<>( false, - UNGROUPED_EXECUTION, - UNGROUPED_EXECUTION, - ignored -> factory, + factory, factory.getOutputTypes()); } private final List buildOutputTypes; private final boolean buildOuter; - private final PipelineExecutionStrategy probeExecutionStrategy; - private final PipelineExecutionStrategy buildExecutionStrategy; - private final Function joinBridgeProvider; - - private final FreezeOnReadCounter probeFactoryCount = new FreezeOnReadCounter(); + private final T joinBridge; private final AtomicBoolean initialized = new AtomicBoolean(); - private InternalJoinBridgeDataManager internalJoinBridgeDataManager; + private JoinLifecycle joinLifecycle; + + private final FreezeOnReadCounter probeFactoryCount = new FreezeOnReadCounter(); public JoinBridgeManager( boolean buildOuter, - PipelineExecutionStrategy probeExecutionStrategy, - PipelineExecutionStrategy lookupSourceExecutionStrategy, - Function lookupSourceFactoryProvider, + T joinBridge, List buildOutputTypes) { this.buildOuter = buildOuter; - this.probeExecutionStrategy = requireNonNull(probeExecutionStrategy, "probeExecutionStrategy is null"); - this.buildExecutionStrategy = requireNonNull(lookupSourceExecutionStrategy, "lookupSourceExecutionStrategy is null"); - this.joinBridgeProvider = requireNonNull(lookupSourceFactoryProvider, "lookupSourceFactoryProvider is null"); + this.joinBridge = requireNonNull(joinBridge, "joinBridge is null"); this.buildOutputTypes = requireNonNull(buildOutputTypes, "buildOutputTypes is null"); } @@ -81,7 +67,7 @@ private void initializeIfNecessary() return; } int finalProbeFactoryCount = probeFactoryCount.get(); - internalJoinBridgeDataManager = internalJoinBridgeDataManager(probeExecutionStrategy, buildExecutionStrategy, joinBridgeProvider, finalProbeFactoryCount, buildOuter ? 1 : 0); + joinLifecycle = new JoinLifecycle(joinBridge, finalProbeFactoryCount, buildOuter ? 1 : 0); initialized.set(true); } } @@ -92,378 +78,57 @@ public List getBuildOutputTypes() return buildOutputTypes; } - public PipelineExecutionStrategy getBuildExecutionStrategy() - { - return buildExecutionStrategy; - } - public void incrementProbeFactoryCount() { probeFactoryCount.increment(); } - public T getJoinBridge(Lifespan lifespan) + public T getJoinBridge() { initializeIfNecessary(); - return internalJoinBridgeDataManager.getJoinBridge(lifespan); + return joinBridge; } - /** - * Invoked when a probe operator factory indicates that it will not - * create any more operators, for any lifespan. - *

- * It is expected that this method will only be invoked after - * {@link #probeOperatorFactoryClosed(Lifespan)} has been invoked - * for every known lifespan. - */ - public void probeOperatorFactoryClosedForAllLifespans() + public void probeOperatorFactoryClosed() { initializeIfNecessary(); - internalJoinBridgeDataManager.probeOperatorFactoryClosedForAllLifespans(); + joinLifecycle.releaseForProbe(); } - public void probeOperatorFactoryClosed(Lifespan lifespan) + public void probeOperatorCreated() { initializeIfNecessary(); - internalJoinBridgeDataManager.probeOperatorFactoryClosed(lifespan); + joinLifecycle.retainForProbe(); } - public void probeOperatorCreated(Lifespan lifespan) + public void probeOperatorClosed() { initializeIfNecessary(); - internalJoinBridgeDataManager.probeOperatorCreated(lifespan); + joinLifecycle.releaseForProbe(); } - public void probeOperatorClosed(Lifespan lifespan) + public void outerOperatorFactoryClosed() { initializeIfNecessary(); - internalJoinBridgeDataManager.probeOperatorClosed(lifespan); + joinLifecycle.releaseForOuter(); } - public void outerOperatorFactoryClosed(Lifespan lifespan) + public void outerOperatorCreated() { initializeIfNecessary(); - internalJoinBridgeDataManager.outerOperatorFactoryClosed(lifespan); + joinLifecycle.retainForOuter(); } - public void outerOperatorCreated(Lifespan lifespan) + public void outerOperatorClosed() { initializeIfNecessary(); - internalJoinBridgeDataManager.outerOperatorCreated(lifespan); + joinLifecycle.releaseForOuter(); } - public void outerOperatorClosed(Lifespan lifespan) + public ListenableFuture getOuterPositionsFuture() { initializeIfNecessary(); - internalJoinBridgeDataManager.outerOperatorClosed(lifespan); - } - - public ListenableFuture getOuterPositionsFuture(Lifespan lifespan) - { - initializeIfNecessary(); - return internalJoinBridgeDataManager.getOuterPositionsFuture(lifespan); - } - - private static InternalJoinBridgeDataManager internalJoinBridgeDataManager( - PipelineExecutionStrategy probeExecutionStrategy, - PipelineExecutionStrategy buildExecutionStrategy, - Function joinBridgeProvider, - int probeFactoryCount, - int outerFactoryCount) - { - checkArgument(outerFactoryCount == 0 || outerFactoryCount == 1, "outerFactoryCount should only be 0 or 1 because it is expected that outer factory never gets duplicated."); - switch (probeExecutionStrategy) { - case UNGROUPED_EXECUTION: - switch (buildExecutionStrategy) { - case UNGROUPED_EXECUTION: - return new TaskWideInternalJoinBridgeDataManager<>(joinBridgeProvider, probeFactoryCount, outerFactoryCount); - case GROUPED_EXECUTION: - throw new UnsupportedOperationException("Invalid combination. Lookup source should not be grouped if probe is not going to take advantage of it."); - } - throw new UnsupportedOperationException("Unknown buildExecutionStrategy: " + buildExecutionStrategy); - - case GROUPED_EXECUTION: - switch (buildExecutionStrategy) { - case UNGROUPED_EXECUTION: - return new SharedInternalJoinBridgeDataManager<>(joinBridgeProvider, probeFactoryCount, outerFactoryCount); - case GROUPED_EXECUTION: - return new OneToOneInternalJoinBridgeDataManager<>(joinBridgeProvider, probeFactoryCount, outerFactoryCount); - } - throw new UnsupportedOperationException("Unknown buildExecutionStrategy: " + buildExecutionStrategy); - } - throw new UnsupportedOperationException("Unknown probeExecutionStrategy: " + probeExecutionStrategy); - } - - private interface InternalJoinBridgeDataManager - { - T getJoinBridge(Lifespan lifespan); - - ListenableFuture getOuterPositionsFuture(Lifespan lifespan); - - void probeOperatorFactoryClosedForAllLifespans(); - - void probeOperatorFactoryClosed(Lifespan lifespan); - - void probeOperatorCreated(Lifespan lifespan); - - void probeOperatorClosed(Lifespan lifespan); - - void outerOperatorFactoryClosed(Lifespan lifespan); - - void outerOperatorCreated(Lifespan lifespan); - - void outerOperatorClosed(Lifespan lifespan); - } - - // 1 probe, 1 lookup source - private static class TaskWideInternalJoinBridgeDataManager - implements InternalJoinBridgeDataManager - { - private final T joinBridge; - private final JoinLifecycle joinLifecycle; - - public TaskWideInternalJoinBridgeDataManager(Function lookupSourceFactoryProvider, int probeFactoryCount, int outerFactoryCount) - { - joinBridge = lookupSourceFactoryProvider.apply(Lifespan.taskWide()); - joinLifecycle = new JoinLifecycle(joinBridge, probeFactoryCount, outerFactoryCount); - } - - @Override - public T getJoinBridge(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - return joinBridge; - } - - @Override - public ListenableFuture getOuterPositionsFuture(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - return transform(joinLifecycle.whenBuildAndProbeFinishes(), ignored -> joinBridge.getOuterPositionIterator(), directExecutor()); - } - - @Override - public void probeOperatorFactoryClosedForAllLifespans() - { - // do nothing - } - - @Override - public void probeOperatorFactoryClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.releaseForProbe(); - } - - @Override - public void probeOperatorCreated(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.retainForProbe(); - } - - @Override - public void probeOperatorClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.releaseForProbe(); - } - - @Override - public void outerOperatorFactoryClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.releaseForOuter(); - } - - @Override - public void outerOperatorCreated(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.retainForOuter(); - } - - @Override - public void outerOperatorClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan)); - joinLifecycle.releaseForOuter(); - } - } - - // N probe, N lookup source; one-to-one mapping, bijective - private static class OneToOneInternalJoinBridgeDataManager - implements InternalJoinBridgeDataManager - { - private final Map> joinBridgeMap = new ConcurrentHashMap<>(); - private final Function joinBridgeProvider; - private final int probeFactoryCount; - private final int outerFactoryCount; - - public OneToOneInternalJoinBridgeDataManager(Function joinBridgeProvider, int probeFactoryCount, int outerFactoryCount) - { - this.joinBridgeProvider = joinBridgeProvider; - this.probeFactoryCount = probeFactoryCount; - this.outerFactoryCount = outerFactoryCount; - } - - @Override - public T getJoinBridge(Lifespan lifespan) - { - return data(lifespan).joinBridge; - } - - @Override - public ListenableFuture getOuterPositionsFuture(Lifespan lifespan) - { - return transform( - data(lifespan).joinLifecycle.whenBuildAndProbeFinishes(), - ignored -> data(lifespan).joinBridge.getOuterPositionIterator(), directExecutor()); - } - - @Override - public void probeOperatorFactoryClosedForAllLifespans() - { - // do nothing - } - - @Override - public void probeOperatorFactoryClosed(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.releaseForProbe(); - } - - @Override - public void probeOperatorCreated(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.retainForProbe(); - } - - @Override - public void probeOperatorClosed(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.releaseForProbe(); - } - - @Override - public void outerOperatorFactoryClosed(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.releaseForOuter(); - } - - @Override - public void outerOperatorCreated(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.retainForOuter(); - } - - @Override - public void outerOperatorClosed(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - data(lifespan).joinLifecycle.releaseForOuter(); - } - - private JoinBridgeAndLifecycle data(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan)); - return joinBridgeMap.computeIfAbsent(lifespan, span -> { - T joinBridge = joinBridgeProvider.apply(span); - return new JoinBridgeAndLifecycle<>(joinBridge, new JoinLifecycle(joinBridge, probeFactoryCount, outerFactoryCount)); - }); - } - - private static class JoinBridgeAndLifecycle - { - T joinBridge; - JoinLifecycle joinLifecycle; - - public JoinBridgeAndLifecycle(T joinBridge, JoinLifecycle joinLifecycle) - { - this.joinBridge = joinBridge; - this.joinLifecycle = joinLifecycle; - } - } - } - - // N probe, 1 lookup source - private static class SharedInternalJoinBridgeDataManager - implements InternalJoinBridgeDataManager - { - private final T taskWideJoinBridge; - - private final JoinLifecycle joinLifecycle; - - public SharedInternalJoinBridgeDataManager(Function lookupSourceFactoryProvider, int probeFactoryCount, int outerFactoryCount) - { - this.taskWideJoinBridge = lookupSourceFactoryProvider.apply(Lifespan.taskWide()); - this.joinLifecycle = new JoinLifecycle(taskWideJoinBridge, probeFactoryCount, outerFactoryCount); - } - - @Override - public T getJoinBridge(Lifespan lifespan) - { - return taskWideJoinBridge; - } - - @Override - public ListenableFuture getOuterPositionsFuture(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan), "join bridge is not partitioned"); - return transform(joinLifecycle.whenBuildAndProbeFinishes(), ignored -> taskWideJoinBridge.getOuterPositionIterator(), directExecutor()); - } - - @Override - public void probeOperatorFactoryClosedForAllLifespans() - { - joinLifecycle.releaseForProbe(); - } - - @Override - public void probeOperatorFactoryClosed(Lifespan lifespan) - { - // do nothing - } - - @Override - public void probeOperatorCreated(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan), "build operator should not produce or destroy probes"); - joinLifecycle.retainForProbe(); - } - - @Override - public void probeOperatorClosed(Lifespan lifespan) - { - checkArgument(!Lifespan.taskWide().equals(lifespan), "build operator should not produce or destroy probes"); - joinLifecycle.releaseForProbe(); - } - - @Override - public void outerOperatorFactoryClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan), "join bridge is not partitioned"); - joinLifecycle.releaseForOuter(); - } - - @Override - public void outerOperatorCreated(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan), "join bridge is not partitioned"); - joinLifecycle.retainForOuter(); - } - - @Override - public void outerOperatorClosed(Lifespan lifespan) - { - checkArgument(Lifespan.taskWide().equals(lifespan), "join bridge is not partitioned"); - joinLifecycle.releaseForOuter(); - } + return transform(joinLifecycle.whenBuildAndProbeFinishes(), ignored -> joinBridge.getOuterPositionIterator(), directExecutor()); } private static class JoinLifecycle diff --git a/core/trino-main/src/main/java/io/trino/operator/join/JoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/join/JoinOperatorFactory.java index 919a70195f90..2ca9360e68cb 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/JoinOperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/JoinOperatorFactory.java @@ -14,36 +14,11 @@ package io.trino.operator.join; import io.trino.operator.OperatorFactory; -import io.trino.operator.PipelineExecutionStrategy; import java.util.Optional; -import static java.util.Objects.requireNonNull; - public interface JoinOperatorFactory extends OperatorFactory { - Optional createOuterOperatorFactory(); - - class OuterOperatorFactoryResult - { - private final OperatorFactory outerOperatorFactory; - private final PipelineExecutionStrategy buildExecutionStrategy; - - public OuterOperatorFactoryResult(OperatorFactory outerOperatorFactory, PipelineExecutionStrategy buildExecutionStrategy) - { - this.outerOperatorFactory = requireNonNull(outerOperatorFactory, "outerOperatorFactory is null"); - this.buildExecutionStrategy = requireNonNull(buildExecutionStrategy, "buildExecutionStrategy is null"); - } - - public OperatorFactory getOuterOperatorFactory() - { - return outerOperatorFactory; - } - - public PipelineExecutionStrategy getBuildExecutionStrategy() - { - return buildExecutionStrategy; - } - } + Optional createOuterOperatorFactory(); } diff --git a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java index a0745c77ae2a..0190e8c7d320 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/LookupJoinOperatorFactory.java @@ -14,12 +14,12 @@ package io.trino.operator.join; import com.google.common.collect.ImmutableList; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; import io.trino.operator.HashGenerator; import io.trino.operator.InterpretedHashGenerator; import io.trino.operator.Operator; import io.trino.operator.OperatorContext; +import io.trino.operator.OperatorFactory; import io.trino.operator.PrecomputedHashGenerator; import io.trino.operator.ProcessorContext; import io.trino.operator.WorkProcessor; @@ -65,7 +65,7 @@ public enum JoinType private final boolean outputSingleMatch; private final boolean waitForBuild; private final JoinProbeFactory joinProbeFactory; - private final Optional outerOperatorFactoryResult; + private final Optional outerOperatorFactory; private final JoinBridgeManager joinBridgeManager; private final OptionalInt totalOperatorsCount; private final HashGenerator probeHashGenerator; @@ -103,17 +103,15 @@ public LookupJoinOperatorFactory( joinBridgeManager.incrementProbeFactoryCount(); if (joinType == INNER || joinType == PROBE_OUTER) { - this.outerOperatorFactoryResult = Optional.empty(); + this.outerOperatorFactory = Optional.empty(); } else { - this.outerOperatorFactoryResult = Optional.of(new OuterOperatorFactoryResult( - new LookupOuterOperatorFactory( - operatorId, - planNodeId, - probeOutputTypes, - buildOutputTypes, - lookupSourceFactoryManager), - lookupSourceFactoryManager.getBuildExecutionStrategy())); + this.outerOperatorFactory = Optional.of(new LookupOuterOperatorFactory( + operatorId, + planNodeId, + probeOutputTypes, + buildOutputTypes, + lookupSourceFactoryManager)); } this.totalOperatorsCount = requireNonNull(totalOperatorsCount, "totalOperatorsCount is null"); @@ -145,8 +143,8 @@ private LookupJoinOperatorFactory(LookupJoinOperatorFactory other) outputSingleMatch = other.outputSingleMatch; waitForBuild = other.waitForBuild; joinProbeFactory = other.joinProbeFactory; + outerOperatorFactory = other.outerOperatorFactory; joinBridgeManager = other.joinBridgeManager; - outerOperatorFactoryResult = other.outerOperatorFactoryResult; totalOperatorsCount = other.totalOperatorsCount; probeHashGenerator = other.probeHashGenerator; partitioningSpillerFactory = other.partitioningSpillerFactory; @@ -156,9 +154,9 @@ private LookupJoinOperatorFactory(LookupJoinOperatorFactory other) } @Override - public Optional createOuterOperatorFactory() + public Optional createOuterOperatorFactory() { - return outerOperatorFactoryResult; + return outerOperatorFactory; } // Methods from OperatorFactory @@ -176,12 +174,6 @@ public void noMoreOperators() close(); } - @Override - public void noMoreOperators(Lifespan lifespan) - { - lifespanFinished(lifespan); - } - // Methods from AdapterWorkProcessorOperatorFactory @Override @@ -206,9 +198,9 @@ public String getOperatorType() public WorkProcessorOperator create(ProcessorContext processorContext, WorkProcessor sourcePages) { checkState(!closed, "Factory is already closed"); - LookupSourceFactory lookupSourceFactory = joinBridgeManager.getJoinBridge(processorContext.getLifespan()); + LookupSourceFactory lookupSourceFactory = joinBridgeManager.getJoinBridge(); - joinBridgeManager.probeOperatorCreated(processorContext.getLifespan()); + joinBridgeManager.probeOperatorCreated(); return new LookupJoinOperator( probeTypes, buildOutputTypes, @@ -217,7 +209,7 @@ public WorkProcessorOperator create(ProcessorContext processorContext, WorkProce waitForBuild, lookupSourceFactory, joinProbeFactory, - () -> joinBridgeManager.probeOperatorClosed(processorContext.getLifespan()), + () -> joinBridgeManager.probeOperatorClosed(), totalOperatorsCount, probeHashGenerator, partitioningSpillerFactory, @@ -229,9 +221,9 @@ public WorkProcessorOperator create(ProcessorContext processorContext, WorkProce public AdapterWorkProcessorOperator createAdapterOperator(ProcessorContext processorContext) { checkState(!closed, "Factory is already closed"); - LookupSourceFactory lookupSourceFactory = joinBridgeManager.getJoinBridge(processorContext.getLifespan()); + LookupSourceFactory lookupSourceFactory = joinBridgeManager.getJoinBridge(); - joinBridgeManager.probeOperatorCreated(processorContext.getLifespan()); + joinBridgeManager.probeOperatorCreated(); return new LookupJoinOperator( probeTypes, buildOutputTypes, @@ -240,7 +232,7 @@ public AdapterWorkProcessorOperator createAdapterOperator(ProcessorContext proce waitForBuild, lookupSourceFactory, joinProbeFactory, - () -> joinBridgeManager.probeOperatorClosed(processorContext.getLifespan()), + () -> joinBridgeManager.probeOperatorClosed(), totalOperatorsCount, probeHashGenerator, partitioningSpillerFactory, @@ -248,18 +240,12 @@ public AdapterWorkProcessorOperator createAdapterOperator(ProcessorContext proce Optional.empty()); } - @Override - public void lifespanFinished(Lifespan lifespan) - { - joinBridgeManager.probeOperatorFactoryClosed(lifespan); - } - @Override public void close() { + joinBridgeManager.probeOperatorFactoryClosed(); checkState(!closed); closed = true; - joinBridgeManager.probeOperatorFactoryClosedForAllLifespans(); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/join/LookupOuterOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/LookupOuterOperator.java index 812f0d518f27..97f5ac8eb73a 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/LookupOuterOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/LookupOuterOperator.java @@ -16,7 +16,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; import io.trino.operator.Operator; import io.trino.operator.OperatorContext; @@ -26,9 +25,7 @@ import io.trino.spi.type.Type; import io.trino.sql.planner.plan.PlanNodeId; -import java.util.HashSet; import java.util.List; -import java.util.Set; import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; @@ -47,7 +44,6 @@ public static class LookupOuterOperatorFactory private final List buildOutputTypes; private final JoinBridgeManager joinBridgeManager; - private final Set createdLifespans = new HashSet<>(); private boolean closed; public LookupOuterOperatorFactory( @@ -73,27 +69,17 @@ public int getOperatorId() public Operator createOperator(DriverContext driverContext) { checkState(!closed, "LookupOuterOperatorFactory is closed"); - Lifespan lifespan = driverContext.getLifespan(); - if (createdLifespans.contains(lifespan)) { - throw new IllegalStateException("Only one outer operator can be created per Lifespan"); - } - createdLifespans.add(lifespan); - ListenableFuture outerPositionsFuture = joinBridgeManager.getOuterPositionsFuture(lifespan); + ListenableFuture outerPositionsFuture = joinBridgeManager.getOuterPositionsFuture(); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, LookupOuterOperator.class.getSimpleName()); - joinBridgeManager.outerOperatorCreated(lifespan); - return new LookupOuterOperator(operatorContext, outerPositionsFuture, probeOutputTypes, buildOutputTypes, () -> joinBridgeManager.outerOperatorClosed(lifespan)); - } - - @Override - public void noMoreOperators(Lifespan lifespan) - { - joinBridgeManager.outerOperatorFactoryClosed(lifespan); + joinBridgeManager.outerOperatorCreated(); + return new LookupOuterOperator(operatorContext, outerPositionsFuture, probeOutputTypes, buildOutputTypes, () -> joinBridgeManager.outerOperatorClosed()); } @Override public void noMoreOperators() { + joinBridgeManager.outerOperatorFactoryClosed(); if (closed) { return; } diff --git a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopBuildOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopBuildOperator.java index 4987f8c14617..3b0950bc787c 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopBuildOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopBuildOperator.java @@ -52,7 +52,7 @@ public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, NestedLoopBuildOperator.class.getSimpleName()); - return new NestedLoopBuildOperator(operatorContext, nestedLoopJoinBridgeManager.getJoinBridge(driverContext.getLifespan())); + return new NestedLoopBuildOperator(operatorContext, nestedLoopJoinBridgeManager.getJoinBridge()); } @Override diff --git a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java index c4a5052ec9b6..a26da4b8e25e 100644 --- a/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java +++ b/core/trino-main/src/main/java/io/trino/operator/join/NestedLoopJoinOperator.java @@ -18,7 +18,6 @@ import com.google.common.primitives.Ints; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; import io.trino.operator.Operator; import io.trino.operator.OperatorContext; @@ -66,7 +65,7 @@ public NestedLoopJoinOperatorFactory( this.operatorId = operatorId; this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); this.joinBridgeManager = nestedLoopJoinBridgeManager; - this.joinBridgeManager.incrementProbeFactoryCount(); + joinBridgeManager.incrementProbeFactoryCount(); this.probeChannels = ImmutableList.copyOf(requireNonNull(probeChannels, "probeChannels is null")); this.buildChannels = ImmutableList.copyOf(requireNonNull(buildChannels, "buildChannels is null")); } @@ -84,7 +83,6 @@ private NestedLoopJoinOperatorFactory(NestedLoopJoinOperatorFactory other) // closed is intentionally not copied closed = false; - joinBridgeManager.incrementProbeFactoryCount(); } @@ -92,33 +90,27 @@ private NestedLoopJoinOperatorFactory(NestedLoopJoinOperatorFactory other) public Operator createOperator(DriverContext driverContext) { checkState(!closed, "Factory is already closed"); - NestedLoopJoinBridge nestedLoopJoinBridge = joinBridgeManager.getJoinBridge(driverContext.getLifespan()); + NestedLoopJoinBridge nestedLoopJoinBridge = joinBridgeManager.getJoinBridge(); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, NestedLoopJoinOperator.class.getSimpleName()); - joinBridgeManager.probeOperatorCreated(driverContext.getLifespan()); + joinBridgeManager.probeOperatorCreated(); return new NestedLoopJoinOperator( operatorContext, nestedLoopJoinBridge, probeChannels, buildChannels, - () -> joinBridgeManager.probeOperatorClosed(driverContext.getLifespan())); + () -> joinBridgeManager.probeOperatorClosed()); } @Override public void noMoreOperators() { + joinBridgeManager.probeOperatorFactoryClosed(); if (closed) { return; } closed = true; - joinBridgeManager.probeOperatorFactoryClosedForAllLifespans(); - } - - @Override - public void noMoreOperators(Lifespan lifespan) - { - joinBridgeManager.probeOperatorFactoryClosed(lifespan); } @Override diff --git a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java index 186c4176de4a..335180e42b3d 100644 --- a/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java +++ b/core/trino-main/src/main/java/io/trino/server/remotetask/HttpRemoteTask.java @@ -36,7 +36,6 @@ import io.trino.execution.DynamicFiltersCollector; import io.trino.execution.DynamicFiltersCollector.VersionedDynamicFilterDomains; import io.trino.execution.FutureStateChange; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap.PartitionedSplitCountTracker; import io.trino.execution.PartitionedSplitsInfo; import io.trino.execution.RemoteTask; @@ -143,8 +142,6 @@ public final class HttpRemoteTask @GuardedBy("this") private volatile long pendingSourceSplitsWeight; @GuardedBy("this") - private final SetMultimap pendingNoMoreSplitsForLifespan = HashMultimap.create(); - @GuardedBy("this") // The keys of this map represent all plan nodes that have "no more splits". // The boolean value of each entry represents whether the "no more splits" notification is pending delivery to workers. private final Map noMoreSplits = new HashMap<>(); @@ -416,14 +413,6 @@ public synchronized void noMoreSplits(PlanNodeId sourceId) triggerUpdate(); } - @Override - public synchronized void noMoreSplits(PlanNodeId sourceId, Lifespan lifespan) - { - if (pendingNoMoreSplitsForLifespan.put(sourceId, lifespan)) { - triggerUpdate(); - } - } - @Override public synchronized void setOutputBuffers(OutputBuffers newOutputBuffers) { @@ -560,9 +549,6 @@ private synchronized void processTaskUpdate(TaskInfo newValue, List splits = pendingSplits.get(planNodeId); boolean pendingNoMoreSplits = Boolean.TRUE.equals(this.noMoreSplits.get(planNodeId)); boolean noMoreSplits = this.noMoreSplits.containsKey(planNodeId); - Set noMoreSplitsForLifespan = pendingNoMoreSplitsForLifespan.get(planNodeId); SplitAssignment assignment = null; - if (!splits.isEmpty() || !noMoreSplitsForLifespan.isEmpty() || pendingNoMoreSplits) { - assignment = new SplitAssignment(planNodeId, splits, noMoreSplitsForLifespan, noMoreSplits); + if (!splits.isEmpty() || pendingNoMoreSplits) { + assignment = new SplitAssignment(planNodeId, splits, noMoreSplits); } return assignment; } diff --git a/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java b/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java index 8bd175b23116..a31d3af68fa9 100644 --- a/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/BufferingSplitSource.java @@ -16,9 +16,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; -import io.trino.spi.connector.ConnectorPartitionHandle; import java.util.ArrayList; import java.util.List; @@ -48,10 +46,10 @@ public CatalogName getCatalogName() } @Override - public ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize) + public ListenableFuture getNextBatch(int maxSize) { checkArgument(maxSize > 0, "Cannot fetch a batch of zero size"); - return GetNextBatch.fetchNextBatchAsync(source, Math.min(bufferSize, maxSize), maxSize, partitionHandle, lifespan); + return GetNextBatch.fetchNextBatchAsync(source, Math.min(bufferSize, maxSize), maxSize); } @Override @@ -77,8 +75,6 @@ private static class GetNextBatch private final SplitSource splitSource; private final int min; private final int max; - private final ConnectorPartitionHandle partitionHandle; - private final Lifespan lifespan; private final List splits = new ArrayList<>(); private boolean noMoreSplits; @@ -86,23 +82,19 @@ private static class GetNextBatch public static ListenableFuture fetchNextBatchAsync( SplitSource splitSource, int min, - int max, - ConnectorPartitionHandle partitionHandle, - Lifespan lifespan) + int max) { - GetNextBatch getNextBatch = new GetNextBatch(splitSource, min, max, partitionHandle, lifespan); + GetNextBatch getNextBatch = new GetNextBatch(splitSource, min, max); ListenableFuture future = getNextBatch.fetchSplits(); return Futures.transform(future, ignored -> new SplitBatch(getNextBatch.splits, getNextBatch.noMoreSplits), directExecutor()); } - private GetNextBatch(SplitSource splitSource, int min, int max, ConnectorPartitionHandle partitionHandle, Lifespan lifespan) + private GetNextBatch(SplitSource splitSource, int min, int max) { this.splitSource = requireNonNull(splitSource, "splitSource is null"); checkArgument(min <= max, "Min splits greater than max splits"); this.min = min; this.max = max; - this.partitionHandle = requireNonNull(partitionHandle, "partitionHandle is null"); - this.lifespan = requireNonNull(lifespan, "lifespan is null"); } private ListenableFuture fetchSplits() @@ -110,7 +102,7 @@ private ListenableFuture fetchSplits() if (splits.size() >= min) { return immediateVoidFuture(); } - ListenableFuture future = splitSource.getNextBatch(partitionHandle, lifespan, max - splits.size()); + ListenableFuture future = splitSource.getNextBatch(max - splits.size()); return Futures.transformAsync(future, splitBatch -> { splits.addAll(splitBatch.getSplits()); if (splitBatch.isLastBatch()) { diff --git a/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java b/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java index 83792cbf5674..ef1e082cec7c 100644 --- a/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/ConnectorAwareSplitSource.java @@ -17,9 +17,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.ConnectorSplitSource.ConnectorSplitBatch; @@ -50,13 +48,13 @@ public CatalogName getCatalogName() } @Override - public ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize) + public ListenableFuture getNextBatch(int maxSize) { - ListenableFuture nextBatch = toListenableFuture(source.getNextBatch(partitionHandle, maxSize)); + ListenableFuture nextBatch = toListenableFuture(source.getNextBatch(maxSize)); return Futures.transform(nextBatch, splitBatch -> { ImmutableList.Builder result = ImmutableList.builder(); for (ConnectorSplit connectorSplit : splitBatch.getSplits()) { - result.add(new Split(catalogName, connectorSplit, lifespan)); + result.add(new Split(catalogName, connectorSplit)); } return new SplitBatch(result.build(), splitBatch.isNoMoreSplits()); }, directExecutor()); diff --git a/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java b/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java index 120ff8d6a079..3ad8772dc258 100644 --- a/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/SampledSplitSource.java @@ -16,8 +16,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; -import io.trino.spi.connector.ConnectorPartitionHandle; import javax.annotation.Nullable; @@ -49,9 +47,9 @@ public CatalogName getCatalogName() } @Override - public ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize) + public ListenableFuture getNextBatch(int maxSize) { - ListenableFuture batch = splitSource.getNextBatch(partitionHandle, lifespan, maxSize); + ListenableFuture batch = splitSource.getNextBatch(maxSize); return Futures.transform(batch, splitBatch -> new SplitBatch( splitBatch.getSplits().stream() .filter(input -> ThreadLocalRandom.current().nextDouble() < sampleRatio) diff --git a/core/trino-main/src/main/java/io/trino/split/SplitManager.java b/core/trino-main/src/main/java/io/trino/split/SplitManager.java index 8b427ac79705..2d7643727330 100644 --- a/core/trino-main/src/main/java/io/trino/split/SplitManager.java +++ b/core/trino-main/src/main/java/io/trino/split/SplitManager.java @@ -20,7 +20,6 @@ import io.trino.metadata.TableHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; -import io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.DynamicFilter; @@ -45,7 +44,6 @@ public SplitManager(CatalogServiceProvider splitManagerPr public SplitSource getSplits( Session session, TableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { @@ -61,7 +59,6 @@ public SplitSource getSplits( table.getTransaction(), connectorSession, table.getConnectorHandle(), - splitSchedulingStrategy, dynamicFilter, constraint); diff --git a/core/trino-main/src/main/java/io/trino/split/SplitSource.java b/core/trino-main/src/main/java/io/trino/split/SplitSource.java index b2247b4f9440..c2de23a2aacc 100644 --- a/core/trino-main/src/main/java/io/trino/split/SplitSource.java +++ b/core/trino-main/src/main/java/io/trino/split/SplitSource.java @@ -15,9 +15,7 @@ import com.google.common.util.concurrent.ListenableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; -import io.trino.spi.connector.ConnectorPartitionHandle; import java.io.Closeable; import java.util.List; @@ -30,7 +28,7 @@ public interface SplitSource { CatalogName getCatalogName(); - ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize); + ListenableFuture getNextBatch(int maxSize); @Override void close(); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java index 819309be1f35..2e3ac288787b 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalExecutionPlanner.java @@ -74,7 +74,6 @@ import io.trino.operator.PagesIndex; import io.trino.operator.PagesSpatialIndexFactory; import io.trino.operator.PartitionFunction; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.RefreshMaterializedViewOperator.RefreshMaterializedViewOperatorFactory; import io.trino.operator.RetryPolicy; import io.trino.operator.RowNumberOperator; @@ -86,7 +85,6 @@ import io.trino.operator.SpatialIndexBuilderOperator.SpatialIndexBuilderOperatorFactory; import io.trino.operator.SpatialIndexBuilderOperator.SpatialPredicate; import io.trino.operator.SpatialJoinOperator.SpatialJoinOperatorFactory; -import io.trino.operator.StageExecutionDescriptor; import io.trino.operator.StatisticsWriterOperator.StatisticsWriterOperatorFactory; import io.trino.operator.StreamingAggregationOperator; import io.trino.operator.TableDeleteOperator.TableDeleteOperatorFactory; @@ -105,7 +103,7 @@ import io.trino.operator.aggregation.DistinctAccumulatorFactory; import io.trino.operator.aggregation.OrderedAccumulatorFactory; import io.trino.operator.aggregation.partial.PartialAggregationController; -import io.trino.operator.exchange.LocalExchange.LocalExchangeFactory; +import io.trino.operator.exchange.LocalExchange; import io.trino.operator.exchange.LocalExchangeSinkOperator.LocalExchangeSinkOperatorFactory; import io.trino.operator.exchange.LocalExchangeSourceOperator.LocalExchangeSourceOperatorFactory; import io.trino.operator.exchange.LocalMergeSourceOperator.LocalMergeSourceOperatorFactory; @@ -119,8 +117,6 @@ import io.trino.operator.join.HashBuilderOperator.HashBuilderOperatorFactory; import io.trino.operator.join.JoinBridgeManager; import io.trino.operator.join.JoinOperatorFactory; -import io.trino.operator.join.JoinOperatorFactory.OuterOperatorFactoryResult; -import io.trino.operator.join.LookupOuterOperator.LookupOuterOperatorFactory; import io.trino.operator.join.LookupSourceFactory; import io.trino.operator.join.NestedLoopJoinBridge; import io.trino.operator.join.NestedLoopJoinPagesSupplier; @@ -303,8 +299,6 @@ import static io.trino.collect.cache.SafeCaches.buildNonEvictableCache; import static io.trino.operator.DistinctLimitOperator.DistinctLimitOperatorFactory; import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier; -import static io.trino.operator.PipelineExecutionStrategy.GROUPED_EXECUTION; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.operator.TableFinishOperator.TableFinishOperatorFactory; import static io.trino.operator.TableFinishOperator.TableFinisher; import static io.trino.operator.TableWriterOperator.FRAGMENT_CHANNEL; @@ -472,7 +466,6 @@ public LocalExecutionPlan plan( PlanNode plan, TypeProvider types, PartitioningScheme partitioningScheme, - StageExecutionDescriptor stageExecutionDescriptor, List partitionedSourceOrder, OutputBuffer outputBuffer) { @@ -483,7 +476,7 @@ public LocalExecutionPlan plan( partitioningScheme.getPartitioning().getHandle().equals(SCALED_WRITER_DISTRIBUTION) || partitioningScheme.getPartitioning().getHandle().equals(SINGLE_DISTRIBUTION) || partitioningScheme.getPartitioning().getHandle().equals(COORDINATOR_DISTRIBUTION)) { - return plan(taskContext, stageExecutionDescriptor, plan, outputLayout, types, partitionedSourceOrder, new TaskOutputFactory(outputBuffer)); + return plan(taskContext, plan, outputLayout, types, partitionedSourceOrder, new TaskOutputFactory(outputBuffer)); } // We can convert the symbols directly into channels, because the root must be a sink and therefore the layout is fixed @@ -534,7 +527,6 @@ public LocalExecutionPlan plan( return plan( taskContext, - stageExecutionDescriptor, plan, outputLayout, types, @@ -552,7 +544,6 @@ public LocalExecutionPlan plan( public LocalExecutionPlan plan( TaskContext taskContext, - StageExecutionDescriptor stageExecutionDescriptor, PlanNode plan, List outputLayout, TypeProvider types, @@ -562,7 +553,7 @@ public LocalExecutionPlan plan( Session session = taskContext.getSession(); LocalExecutionPlanContext context = new LocalExecutionPlanContext(taskContext, types); - PhysicalOperation physicalOperation = plan.accept(new Visitor(session, stageExecutionDescriptor), context); + PhysicalOperation physicalOperation = plan.accept(new Visitor(session), context); Function pagePreprocessor = enforceLoadedLayoutProcessor(outputLayout, physicalOperation.getLayout()); @@ -591,7 +582,7 @@ public LocalExecutionPlan plan( .map(LocalPlannerAware.class::cast) .forEach(LocalPlannerAware::localPlannerComplete); - return new LocalExecutionPlan(context.getDriverFactories(), partitionedSourceOrder, stageExecutionDescriptor); + return new LocalExecutionPlan(context.getDriverFactories(), partitionedSourceOrder); } private static class LocalExecutionPlanContext @@ -635,7 +626,6 @@ private LocalExecutionPlanContext( public void addDriverFactory(boolean inputDriver, boolean outputDriver, PhysicalOperation physicalOperation, OptionalInt driverInstances) { List operatorFactoriesWithTypes = physicalOperation.getOperatorFactoriesWithTypes(); - validateFirstOperatorFactory(inputDriver, operatorFactoriesWithTypes.get(0).getOperatorFactory(), physicalOperation.getPipelineExecutionStrategy()); addLookupOuterDrivers(outputDriver, toOperatorFactories(operatorFactoriesWithTypes)); List operatorFactories; if (isLateMaterializationEnabled(taskContext.getSession())) { @@ -644,7 +634,7 @@ public void addDriverFactory(boolean inputDriver, boolean outputDriver, Physical else { operatorFactories = toOperatorFactories(operatorFactoriesWithTypes); } - driverFactories.add(new DriverFactory(getNextPipelineId(), inputDriver, outputDriver, operatorFactories, driverInstances, physicalOperation.getPipelineExecutionStrategy())); + driverFactories.add(new DriverFactory(getNextPipelineId(), inputDriver, outputDriver, operatorFactories, driverInstances)); } private List handleLateMaterialization(List operatorFactories) @@ -666,38 +656,25 @@ private void addLookupOuterDrivers(boolean isOutputDriver, List } JoinOperatorFactory lookupJoin = (JoinOperatorFactory) operatorFactory; - Optional outerOperatorFactoryResult = lookupJoin.createOuterOperatorFactory(); + Optional outerOperatorFactoryResult = lookupJoin.createOuterOperatorFactory(); if (outerOperatorFactoryResult.isPresent()) { // Add a new driver to output the unmatched rows in an outer join. // We duplicate all of the factories above the JoinOperator (the ones reading from the joins), // and replace the JoinOperator with the OuterOperator (the one that produces unmatched rows). ImmutableList.Builder newOperators = ImmutableList.builder(); - newOperators.add(outerOperatorFactoryResult.get().getOuterOperatorFactory()); + newOperators.add(outerOperatorFactoryResult.get()); operatorFactories.subList(i + 1, operatorFactories.size()).stream() .map(OperatorFactory::duplicate) .forEach(newOperators::add); - addDriverFactory(false, isOutputDriver, newOperators.build(), OptionalInt.of(1), outerOperatorFactoryResult.get().getBuildExecutionStrategy()); + addDriverFactory(false, isOutputDriver, newOperators.build(), OptionalInt.of(1)); } } } - private void addDriverFactory(boolean inputDriver, boolean outputDriver, List operatorFactories, OptionalInt driverInstances, PipelineExecutionStrategy pipelineExecutionStrategy) + private void addDriverFactory(boolean inputDriver, boolean outputDriver, List operatorFactories, OptionalInt driverInstances) { - validateFirstOperatorFactory(inputDriver, operatorFactories.get(0), pipelineExecutionStrategy); - driverFactories.add(new DriverFactory(getNextPipelineId(), inputDriver, outputDriver, operatorFactories, driverInstances, pipelineExecutionStrategy)); - } - - private void validateFirstOperatorFactory(boolean inputDriver, OperatorFactory firstOperatorFactory, PipelineExecutionStrategy pipelineExecutionStrategy) - { - if (pipelineExecutionStrategy == GROUPED_EXECUTION) { - if (inputDriver) { - checkArgument(firstOperatorFactory instanceof ScanFilterAndProjectOperatorFactory || firstOperatorFactory instanceof TableScanOperatorFactory); - } - else { - checkArgument(firstOperatorFactory instanceof LocalExchangeSourceOperatorFactory || firstOperatorFactory instanceof LookupOuterOperatorFactory); - } - } + driverFactories.add(new DriverFactory(getNextPipelineId(), inputDriver, outputDriver, operatorFactories, driverInstances)); } private List getDriverFactories() @@ -822,13 +799,11 @@ public static class LocalExecutionPlan { private final List driverFactories; private final List partitionedSourceOrder; - private final StageExecutionDescriptor stageExecutionDescriptor; - public LocalExecutionPlan(List driverFactories, List partitionedSourceOrder, StageExecutionDescriptor stageExecutionDescriptor) + public LocalExecutionPlan(List driverFactories, List partitionedSourceOrder) { this.driverFactories = ImmutableList.copyOf(requireNonNull(driverFactories, "driverFactories is null")); this.partitionedSourceOrder = ImmutableList.copyOf(requireNonNull(partitionedSourceOrder, "partitionedSourceOrder is null")); - this.stageExecutionDescriptor = requireNonNull(stageExecutionDescriptor, "stageExecutionDescriptor is null"); } public List getDriverFactories() @@ -840,11 +815,6 @@ public List getPartitionedSourceOrder() { return partitionedSourceOrder; } - - public StageExecutionDescriptor getStageExecutionDescriptor() - { - return stageExecutionDescriptor; - } } public static class OperatorFactoryWithTypes @@ -873,12 +843,10 @@ private class Visitor extends PlanVisitor { private final Session session; - private final StageExecutionDescriptor stageExecutionDescriptor; - private Visitor(Session session, StageExecutionDescriptor stageExecutionDescriptor) + private Visitor(Session session) { this.session = session; - this.stageExecutionDescriptor = stageExecutionDescriptor; } @Override @@ -920,7 +888,7 @@ private PhysicalOperation createMergeSource(RemoteSourceNode node, LocalExecutio sortChannels, sortOrder); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } private PhysicalOperation createRemoteSource(RemoteSourceNode node, LocalExecutionPlanContext context) @@ -937,7 +905,7 @@ private PhysicalOperation createRemoteSource(RemoteSourceNode node, LocalExecuti node.getRetryPolicy(), exchangeManagerRegistry); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } @Override @@ -1970,7 +1938,7 @@ else if (sourceNode instanceof SampleNode) { getFilterAndProjectMinOutputPageSize(session), getFilterAndProjectMinOutputPageRowCount(session)); - return new PhysicalOperation(operatorFactory, outputMappings, context, stageExecutionDescriptor.isScanGroupedExecution(sourceNode.getId()) ? GROUPED_EXECUTION : UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, outputMappings, context); } else { Supplier pageProcessor = expressionCompiler.compilePageProcessor(translatedFilter, translatedProjections, Optional.of(context.getStageId() + "_" + planNodeId)); @@ -2018,7 +1986,7 @@ private PhysicalOperation visitTableScan(TableScanNode node, Expression filterEx DynamicFilter dynamicFilter = getDynamicFilter(node, filterExpression, context); OperatorFactory operatorFactory = new TableScanOperatorFactory(context.getNextOperatorId(), node.getId(), pageSourceProvider, node.getTable(), columns, dynamicFilter); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, stageExecutionDescriptor.isScanGroupedExecution(node.getId()) ? GROUPED_EXECUTION : UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } private Optional getStaticFilter(Expression filterExpression) @@ -2059,7 +2027,7 @@ public PhysicalOperation visitValues(ValuesNode node, LocalExecutionPlanContext if (node.getRowCount() == 0) { OperatorFactory operatorFactory = new ValuesOperatorFactory(context.getNextOperatorId(), node.getId(), ImmutableList.of()); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } List outputTypes = getSymbolTypes(node.getOutputSymbols(), context.getTypes()); @@ -2082,7 +2050,7 @@ public PhysicalOperation visitValues(ValuesNode node, LocalExecutionPlanContext } OperatorFactory operatorFactory = new ValuesOperatorFactory(context.getNextOperatorId(), node.getId(), ImmutableList.of(pageBuilder.build())); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } @Override @@ -2198,7 +2166,7 @@ public PhysicalOperation visitIndexSource(IndexSourceNode node, LocalExecutionPl ConnectorIndex index = indexManager.getIndex(session, node.getIndexHandle(), lookupSchema, outputSchema); OperatorFactory operatorFactory = new IndexSourceOperator.IndexSourceOperatorFactory(context.getNextOperatorId(), node.getId(), index, probeKeyNormalizer); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } /** @@ -2315,16 +2283,10 @@ public PhysicalOperation visitIndexJoin(IndexJoinNode node, LocalExecutionPlanCo joinCompiler, blockTypeOperators); - verify(probeSource.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION); - verify(indexSource.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION); + indexLookupSourceFactory.setTaskContext(context.taskContext); JoinBridgeManager lookupSourceFactoryManager = new JoinBridgeManager<>( false, - UNGROUPED_EXECUTION, - UNGROUPED_EXECUTION, - lifespan -> { - indexLookupSourceFactory.setTaskContext(context.taskContext); - return indexLookupSourceFactory; - }, + indexLookupSourceFactory, indexLookupSourceFactory.getOutputTypes()); ImmutableMap.Builder outputMappings = ImmutableMap.builder(); @@ -2535,16 +2497,11 @@ private PhysicalOperation createNestedLoopJoin(JoinNode node, Set nestedLoopJoinBridgeManager = new JoinBridgeManager<>( false, - probeSource.getPipelineExecutionStrategy(), - buildSource.getPipelineExecutionStrategy(), - lifespan -> new NestedLoopJoinPagesSupplier(), + new NestedLoopJoinPagesSupplier(), buildSource.getTypes()); NestedLoopBuildOperatorFactory nestedLoopBuildOperatorFactory = new NestedLoopBuildOperatorFactory( buildContext.getNextOperatorId(), @@ -2726,7 +2683,6 @@ private PhysicalOperation createLookupJoin( boolean buildOuter = node.getType() == RIGHT || node.getType() == FULL; boolean spillEnabled = isSpillEnabled(session) && node.isSpillable().orElseThrow(() -> new IllegalArgumentException("spillable not yet set")) - && probeSource.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION && !buildOuter; JoinBridgeManager lookupSourceFactory = createLookupSourceFactory(node, buildNode, buildSymbols, buildHashSymbol, probeSource, context, spillEnabled, localDynamicFilters); @@ -2764,12 +2720,6 @@ private JoinBridgeManager createLookupSourceFact LocalExecutionPlanContext buildContext = context.createSubContext(); PhysicalOperation buildSource = buildNode.accept(this, buildContext); - if (buildSource.getPipelineExecutionStrategy() == GROUPED_EXECUTION) { - checkState( - probeSource.getPipelineExecutionStrategy() == GROUPED_EXECUTION, - "Build execution is GROUPED_EXECUTION. Probe execution is expected be GROUPED_EXECUTION, but is UNGROUPED_EXECUTION."); - } - List buildOutputChannels = ImmutableList.copyOf(getChannelsForSymbols(node.getRightOutputSymbols(), buildSource.getLayout())); List buildChannels = ImmutableList.copyOf(getChannelsForSymbols(buildSymbols, buildSource.getLayout())); OptionalInt buildHashChannel = buildHashSymbol.map(channelGetter(buildSource)) @@ -2813,9 +2763,7 @@ private JoinBridgeManager createLookupSourceFact List buildTypes = buildSource.getTypes(); JoinBridgeManager lookupSourceFactoryManager = new JoinBridgeManager<>( buildOuter, - probeSource.getPipelineExecutionStrategy(), - buildSource.getPipelineExecutionStrategy(), - lifespan -> new PartitionedLookupSourceFactory( + new PartitionedLookupSourceFactory( buildTypes, buildOutputTypes, buildChannels.stream() @@ -2928,9 +2876,6 @@ private Optional createDynamicFilter( if (collectedDynamicFilters.isEmpty()) { return Optional.empty(); } - checkState( - buildSource.getPipelineExecutionStrategy() != GROUPED_EXECUTION, - "Dynamic filtering cannot be used with grouped execution"); log.debug("[Join] Dynamic filters: %s", node.getDynamicFilters()); ImmutableList.Builder>> collectors = ImmutableList.builder(); if (!localDynamicFilters.isEmpty()) { @@ -3086,7 +3031,6 @@ public PhysicalOperation visitSemiJoin(SemiJoinNode node, LocalExecutionPlanCont // Plan build LocalExecutionPlanContext buildContext = context.createSubContext(); PhysicalOperation buildSource = node.getFilteringSource().accept(this, buildContext); - checkState(buildSource.getPipelineExecutionStrategy() == probeSource.getPipelineExecutionStrategy(), "build and probe have different pipelineExecutionStrategy"); int partitionCount = buildContext.getDriverInstanceCount().orElse(1); checkArgument(partitionCount == 1, "Expected local execution to not be parallel"); @@ -3170,7 +3114,7 @@ public PhysicalOperation visitRefreshMaterializedView(RefreshMaterializedViewNod { context.setDriverInstanceCount(1); OperatorFactory operatorFactory = new RefreshMaterializedViewOperatorFactory(context.getNextOperatorId(), node.getId(), metadata, node.getViewName()); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } @Override @@ -3374,7 +3318,7 @@ public PhysicalOperation visitSimpleTableExecuteNode(SimpleTableExecuteNode node session, node.getExecuteHandle()); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } @Override @@ -3412,7 +3356,7 @@ public PhysicalOperation visitTableDelete(TableDeleteNode node, LocalExecutionPl { OperatorFactory operatorFactory = new TableDeleteOperatorFactory(context.getNextOperatorId(), node.getId(), metadata, session, node.getTarget()); - return new PhysicalOperation(operatorFactory, makeLayout(node), context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, makeLayout(node), context); } @Override @@ -3467,15 +3411,14 @@ private PhysicalOperation createLocalMerge(ExchangeNode node, LocalExecutionPlan int operatorsCount = subContext.getDriverInstanceCount().orElse(1); List types = getSourceOperatorTypes(node, context.getTypes()); - LocalExchangeFactory exchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, session, - node.getPartitioningScheme().getPartitioning().getHandle(), operatorsCount, - types, + node.getPartitioningScheme().getPartitioning().getHandle(), ImmutableList.of(), + types, Optional.empty(), - source.getPipelineExecutionStrategy(), maxLocalExchangeBufferSize, blockTypeOperators); @@ -3486,10 +3429,9 @@ private PhysicalOperation createLocalMerge(ExchangeNode node, LocalExecutionPlan false, new PhysicalOperation( new LocalExchangeSinkOperatorFactory( - exchangeFactory, + localExchange.createSinkFactory(), subContext.getNextOperatorId(), node.getId(), - exchangeFactory.newSinkFactoryId(), pagePreprocessor), source), subContext.getDriverInstanceCount()); @@ -3503,12 +3445,12 @@ private PhysicalOperation createLocalMerge(ExchangeNode node, LocalExecutionPlan OperatorFactory operatorFactory = new LocalMergeSourceOperatorFactory( context.getNextOperatorId(), node.getId(), - exchangeFactory, + localExchange, types, orderingCompiler, sortChannels, orderings); - return new PhysicalOperation(operatorFactory, layout, context, UNGROUPED_EXECUTION); + return new PhysicalOperation(operatorFactory, layout, context); } private PhysicalOperation createLocalExchange(ExchangeNode node, LocalExecutionPlanContext context) @@ -3533,7 +3475,6 @@ else if (context.getDriverInstanceCount().isPresent()) { Optional hashChannel = node.getPartitioningScheme().getHashColumn() .map(symbol -> node.getOutputSymbols().indexOf(symbol)); - PipelineExecutionStrategy exchangeSourcePipelineExecutionStrategy = GROUPED_EXECUTION; List driverFactoryParametersList = new ArrayList<>(); for (int i = 0; i < node.getSources().size(); i++) { PlanNode sourceNode = node.getSources().get(i); @@ -3541,21 +3482,16 @@ else if (context.getDriverInstanceCount().isPresent()) { LocalExecutionPlanContext subContext = context.createSubContext(); PhysicalOperation source = sourceNode.accept(this, subContext); driverFactoryParametersList.add(new DriverFactoryParameters(subContext, source)); - - if (source.getPipelineExecutionStrategy() == UNGROUPED_EXECUTION) { - exchangeSourcePipelineExecutionStrategy = UNGROUPED_EXECUTION; - } } - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, session, - node.getPartitioningScheme().getPartitioning().getHandle(), driverInstanceCount, - types, + node.getPartitioningScheme().getPartitioning().getHandle(), channels, + types, hashChannel, - exchangeSourcePipelineExecutionStrategy, maxLocalExchangeBufferSize, blockTypeOperators); for (int i = 0; i < node.getSources().size(); i++) { @@ -3571,10 +3507,9 @@ else if (context.getDriverInstanceCount().isPresent()) { false, new PhysicalOperation( new LocalExchangeSinkOperatorFactory( - localExchangeFactory, + localExchange.createSinkFactory(), subContext.getNextOperatorId(), node.getId(), - localExchangeFactory.newSinkFactoryId(), pagePreprocessor), source), subContext.getDriverInstanceCount()); @@ -3584,10 +3519,10 @@ else if (context.getDriverInstanceCount().isPresent()) { context.setInputDriver(false); // instance count must match the number of partitions in the exchange - verify(context.getDriverInstanceCount().getAsInt() == localExchangeFactory.getBufferCount(), + verify(context.getDriverInstanceCount().getAsInt() == localExchange.getBufferCount(), "driver instance count must match the number of exchange partitions"); - return new PhysicalOperation(new LocalExchangeSourceOperatorFactory(context.getNextOperatorId(), node.getId(), localExchangeFactory), makeLayout(node), context, exchangeSourcePipelineExecutionStrategy); + return new PhysicalOperation(new LocalExchangeSourceOperatorFactory(context.getNextOperatorId(), node.getId(), localExchange), makeLayout(node), context); } @Override @@ -4083,35 +4018,31 @@ private static class PhysicalOperation private final Map layout; private final List types; - private final PipelineExecutionStrategy pipelineExecutionStrategy; - - public PhysicalOperation(OperatorFactory operatorFactory, Map layout, LocalExecutionPlanContext context, PipelineExecutionStrategy pipelineExecutionStrategy) + public PhysicalOperation(OperatorFactory operatorFactory, Map layout, LocalExecutionPlanContext context) { - this(operatorFactory, layout, context.getTypes(), Optional.empty(), pipelineExecutionStrategy); + this(operatorFactory, layout, context.getTypes(), Optional.empty()); } public PhysicalOperation(OperatorFactory operatorFactory, Map layout, LocalExecutionPlanContext context, PhysicalOperation source) { - this(operatorFactory, layout, context.getTypes(), Optional.of(requireNonNull(source, "source is null")), source.getPipelineExecutionStrategy()); + this(operatorFactory, layout, context.getTypes(), Optional.of(requireNonNull(source, "source is null"))); } public PhysicalOperation(OperatorFactory outputOperatorFactory, PhysicalOperation source) { - this(outputOperatorFactory, ImmutableMap.of(), TypeProvider.empty(), Optional.of(requireNonNull(source, "source is null")), source.getPipelineExecutionStrategy()); + this(outputOperatorFactory, ImmutableMap.of(), TypeProvider.empty(), Optional.of(requireNonNull(source, "source is null"))); } private PhysicalOperation( OperatorFactory operatorFactory, Map layout, TypeProvider typeProvider, - Optional source, - PipelineExecutionStrategy pipelineExecutionStrategy) + Optional source) { requireNonNull(operatorFactory, "operatorFactory is null"); requireNonNull(layout, "layout is null"); requireNonNull(typeProvider, "typeProvider is null"); requireNonNull(source, "source is null"); - requireNonNull(pipelineExecutionStrategy, "pipelineExecutionStrategy is null"); this.types = toTypes(layout, typeProvider); this.operatorFactoriesWithTypes = ImmutableList.builder() @@ -4119,7 +4050,6 @@ private PhysicalOperation( .add(new OperatorFactoryWithTypes(operatorFactory, types)) .build(); this.layout = ImmutableMap.copyOf(layout); - this.pipelineExecutionStrategy = pipelineExecutionStrategy; } private static List toTypes(Map layout, TypeProvider typeProvider) @@ -4162,11 +4092,6 @@ private List getOperatorFactoriesWithTypes() { return operatorFactoriesWithTypes; } - - public PipelineExecutionStrategy getPipelineExecutionStrategy() - { - return pipelineExecutionStrategy; - } } private static class DriverFactoryParameters diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java index 8b974d3e905a..c9a375213836 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/NodePartitioningManager.java @@ -29,7 +29,6 @@ import io.trino.spi.connector.BucketFunction; import io.trino.spi.connector.ConnectorBucketNodeMap; import io.trino.spi.connector.ConnectorNodePartitioningProvider; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.type.Type; import io.trino.split.EmptySplit; @@ -103,19 +102,6 @@ public BucketFunction getBucketFunction(Session session, PartitioningHandle part return bucketFunction; } - public List listPartitionHandles( - Session session, - PartitioningHandle partitioningHandle) - { - CatalogName catalogName = partitioningHandle.getConnectorId() - .orElseThrow(() -> new IllegalArgumentException("No connector ID for partitioning handle: " + partitioningHandle)); - ConnectorNodePartitioningProvider partitioningProvider = getPartitioningProvider(catalogName); - return partitioningProvider.listPartitionHandles( - partitioningHandle.getTransactionHandle().orElseThrow(() -> new IllegalArgumentException("No transactionHandle for partitioning handle: " + partitioningHandle)), - session.toConnectorSession(catalogName), - partitioningHandle.getConnectorHandle()); - } - public NodePartitionMap getNodePartitioningMap(Session session, PartitioningHandle partitioningHandle) { requireNonNull(session, "session is null"); @@ -217,14 +203,11 @@ private ToIntFunction getSplitToBucket(Session session, PartitioningHandl return split -> { int bucket; if (split.getConnectorSplit() instanceof EmptySplit) { - bucket = split.getLifespan().isTaskWide() ? 0 : split.getLifespan().getId(); + bucket = 0; } else { bucket = splitBucketFunction.applyAsInt(split.getConnectorSplit()); } - if (!split.getLifespan().isTaskWide()) { - checkArgument(split.getLifespan().getId() == bucket); - } return bucket; }; } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java index 3bb3fb282e0b..57990a12b5c0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragment.java @@ -18,7 +18,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import io.trino.cost.StatsAndCosts; -import io.trino.operator.StageExecutionDescriptor; import io.trino.spi.type.Type; import io.trino.sql.planner.plan.PlanFragmentId; import io.trino.sql.planner.plan.PlanNode; @@ -50,7 +49,6 @@ public class PlanFragment private final Set partitionedSourceNodes; private final List remoteSourceNodes; private final PartitioningScheme partitioningScheme; - private final StageExecutionDescriptor stageExecutionDescriptor; private final StatsAndCosts statsAndCosts; private final Optional jsonRepresentation; @@ -66,7 +64,6 @@ private PlanFragment( Set partitionedSourceNodes, List remoteSourceNodes, PartitioningScheme partitioningScheme, - StageExecutionDescriptor stageExecutionDescriptor, StatsAndCosts statsAndCosts) { this.id = requireNonNull(id, "id is null"); @@ -79,7 +76,6 @@ private PlanFragment( this.partitionedSourceNodes = requireNonNull(partitionedSourceNodes, "partitionedSourceNodes is null"); this.remoteSourceNodes = requireNonNull(remoteSourceNodes, "remoteSourceNodes is null"); this.partitioningScheme = requireNonNull(partitioningScheme, "partitioningScheme is null"); - this.stageExecutionDescriptor = requireNonNull(stageExecutionDescriptor, "stageExecutionDescriptor is null"); this.statsAndCosts = requireNonNull(statsAndCosts, "statsAndCosts is null"); this.jsonRepresentation = Optional.empty(); } @@ -92,7 +88,6 @@ public PlanFragment( @JsonProperty("partitioning") PartitioningHandle partitioning, @JsonProperty("partitionedSources") List partitionedSources, @JsonProperty("partitioningScheme") PartitioningScheme partitioningScheme, - @JsonProperty("stageExecutionDescriptor") StageExecutionDescriptor stageExecutionDescriptor, @JsonProperty("statsAndCosts") StatsAndCosts statsAndCosts, @JsonProperty("jsonRepresentation") Optional jsonRepresentation) { @@ -102,7 +97,6 @@ public PlanFragment( this.partitioning = requireNonNull(partitioning, "partitioning is null"); this.partitionedSources = ImmutableList.copyOf(requireNonNull(partitionedSources, "partitionedSources is null")); this.partitionedSourcesSet = ImmutableSet.copyOf(partitionedSources); - this.stageExecutionDescriptor = requireNonNull(stageExecutionDescriptor, "stageExecutionDescriptor is null"); this.statsAndCosts = requireNonNull(statsAndCosts, "statsAndCosts is null"); this.jsonRepresentation = requireNonNull(jsonRepresentation, "jsonRepresentation is null"); @@ -164,12 +158,6 @@ public PartitioningScheme getPartitioningScheme() return partitioningScheme; } - @JsonProperty - public StageExecutionDescriptor getStageExecutionDescriptor() - { - return stageExecutionDescriptor; - } - @JsonProperty public StatsAndCosts getStatsAndCosts() { @@ -200,7 +188,6 @@ public PlanFragment withoutEmbeddedJsonRepresentation() this.partitionedSourceNodes, this.remoteSourceNodes, this.partitioningScheme, - this.stageExecutionDescriptor, this.statsAndCosts); } @@ -255,17 +242,7 @@ private static void findRemoteSourceNodes(PlanNode node, ImmutableList.Builder bucketToPartition) { - return new PlanFragment(id, root, symbols, partitioning, partitionedSources, partitioningScheme.withBucketToPartition(bucketToPartition), stageExecutionDescriptor, statsAndCosts, jsonRepresentation); - } - - public PlanFragment withFixedLifespanScheduleGroupedExecution(List capableTableScanNodes) - { - return new PlanFragment(id, root, symbols, partitioning, partitionedSources, partitioningScheme, StageExecutionDescriptor.fixedLifespanScheduleGroupedExecution(capableTableScanNodes), statsAndCosts, jsonRepresentation); - } - - public PlanFragment withDynamicLifespanScheduleGroupedExecution(List capableTableScanNodes) - { - return new PlanFragment(id, root, symbols, partitioning, partitionedSources, partitioningScheme, StageExecutionDescriptor.dynamicLifespanScheduleGroupedExecution(capableTableScanNodes), statsAndCosts, jsonRepresentation); + return new PlanFragment(id, root, symbols, partitioning, partitionedSources, partitioningScheme.withBucketToPartition(bucketToPartition), statsAndCosts, jsonRepresentation); } @Override diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragmenter.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragmenter.java index d346b1eb8bba..35437e91b1af 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragmenter.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanFragmenter.java @@ -17,10 +17,8 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import io.trino.Session; -import io.trino.SystemSessionProperties; import io.trino.cost.StatsAndCosts; import io.trino.execution.QueryManagerConfig; -import io.trino.execution.scheduler.BucketNodeMap; import io.trino.execution.warnings.WarningCollector; import io.trino.metadata.FunctionManager; import io.trino.metadata.Metadata; @@ -29,22 +27,16 @@ import io.trino.operator.RetryPolicy; import io.trino.spi.TrinoException; import io.trino.spi.TrinoWarning; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.type.Type; -import io.trino.sql.planner.plan.AggregationNode; import io.trino.sql.planner.plan.ExchangeNode; import io.trino.sql.planner.plan.ExplainAnalyzeNode; -import io.trino.sql.planner.plan.JoinNode; import io.trino.sql.planner.plan.OutputNode; -import io.trino.sql.planner.plan.PatternRecognitionNode; import io.trino.sql.planner.plan.PlanFragmentId; import io.trino.sql.planner.plan.PlanNode; import io.trino.sql.planner.plan.PlanNodeId; -import io.trino.sql.planner.plan.PlanVisitor; import io.trino.sql.planner.plan.RefreshMaterializedViewNode; import io.trino.sql.planner.plan.RemoteSourceNode; -import io.trino.sql.planner.plan.RowNumberNode; import io.trino.sql.planner.plan.SimplePlanRewriter; import io.trino.sql.planner.plan.SimpleTableExecuteNode; import io.trino.sql.planner.plan.StatisticsWriterNode; @@ -52,9 +44,7 @@ import io.trino.sql.planner.plan.TableFinishNode; import io.trino.sql.planner.plan.TableScanNode; import io.trino.sql.planner.plan.TableWriterNode; -import io.trino.sql.planner.plan.TopNRankingNode; import io.trino.sql.planner.plan.ValuesNode; -import io.trino.sql.planner.plan.WindowNode; import javax.inject.Inject; @@ -69,21 +59,16 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Predicates.in; import static com.google.common.collect.ImmutableList.toImmutableList; -import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.SystemSessionProperties.getQueryMaxStageCount; import static io.trino.SystemSessionProperties.getRetryPolicy; -import static io.trino.SystemSessionProperties.isDynamicScheduleForGroupedExecution; import static io.trino.SystemSessionProperties.isForceSingleNodeOutput; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.StandardErrorCode.QUERY_HAS_TOO_MANY_STAGES; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.connector.StandardWarningCode.TOO_MANY_STAGES; import static io.trino.sql.planner.SchedulingOrderVisitor.scheduleOrder; import static io.trino.sql.planner.SystemPartitioningHandle.COORDINATOR_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; import static io.trino.sql.planner.plan.ExchangeNode.Scope.REMOTE; -import static io.trino.sql.planner.plan.ExchangeNode.Type.REPLICATE; import static io.trino.sql.planner.planprinter.PlanPrinter.jsonFragmentPlan; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -99,19 +84,16 @@ public class PlanFragmenter private final Metadata metadata; private final FunctionManager functionManager; - private final NodePartitioningManager nodePartitioningManager; private final QueryManagerConfig config; @Inject public PlanFragmenter( Metadata metadata, FunctionManager functionManager, - NodePartitioningManager nodePartitioningManager, QueryManagerConfig queryManagerConfig) { this.metadata = requireNonNull(metadata, "metadata is null"); this.functionManager = requireNonNull(functionManager, "functionManager is null"); - this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null"); this.config = requireNonNull(queryManagerConfig, "queryManagerConfig is null"); } @@ -127,7 +109,6 @@ public SubPlan createSubPlans(Session session, Plan plan, boolean forceSingleNod SubPlan subPlan = fragmenter.buildRootFragment(root, properties); subPlan = reassignPartitioningHandleIfNecessary(session, subPlan); - subPlan = analyzeGroupedExecution(session, subPlan); checkState(!isForceSingleNodeOutput(session) || subPlan.getFragment().getPartitioning().isSingleNode(), "Root of PlanFragment is not single node"); @@ -157,28 +138,6 @@ private void sanityCheckFragmentedPlan(SubPlan subPlan, WarningCollector warning } } - private SubPlan analyzeGroupedExecution(Session session, SubPlan subPlan) - { - PlanFragment fragment = subPlan.getFragment(); - GroupedExecutionProperties properties = fragment.getRoot().accept(new GroupedExecutionTagger(session, metadata, nodePartitioningManager), null); - if (properties.isSubTreeUseful()) { - boolean preferDynamic = fragment.getRemoteSourceNodes().stream().allMatch(node -> node.getExchangeType() == REPLICATE) - && isDynamicScheduleForGroupedExecution(session); - BucketNodeMap bucketNodeMap = nodePartitioningManager.getBucketNodeMap(session, fragment.getPartitioning(), preferDynamic); - if (bucketNodeMap.isDynamic()) { - fragment = fragment.withDynamicLifespanScheduleGroupedExecution(properties.getCapableTableScanNodes()); - } - else { - fragment = fragment.withFixedLifespanScheduleGroupedExecution(properties.getCapableTableScanNodes()); - } - } - ImmutableList.Builder result = ImmutableList.builder(); - for (SubPlan child : subPlan.getChildren()) { - result.add(analyzeGroupedExecution(session, child)); - } - return new SubPlan(fragment, result.build()); - } - private SubPlan reassignPartitioningHandleIfNecessary(Session session, SubPlan subPlan) { return reassignPartitioningHandleIfNecessaryHelper(session, subPlan, subPlan.getFragment().getPartitioning()); @@ -212,7 +171,6 @@ private SubPlan reassignPartitioningHandleIfNecessaryHelper(Session session, Sub outputPartitioningScheme.getHashColumn(), outputPartitioningScheme.isReplicateNullsAndAny(), outputPartitioningScheme.getBucketToPartition()), - fragment.getStageExecutionDescriptor(), fragment.getStatsAndCosts(), fragment.getJsonRepresentation()); @@ -271,7 +229,6 @@ private SubPlan buildFragment(PlanNode root, FragmentProperties properties, Plan properties.getPartitioningHandle(), schedulingOrder, properties.getPartitioningScheme(), - ungroupedExecution(), statsAndCosts.getForSubplan(root), Optional.of(jsonFragmentPlan(root, symbols, metadata, functionManager, session))); @@ -584,238 +541,6 @@ public Set getPartitionedSources() } } - private static class GroupedExecutionTagger - extends PlanVisitor - { - private final Session session; - private final Metadata metadata; - private final NodePartitioningManager nodePartitioningManager; - private final boolean groupedExecutionEnabled; - - public GroupedExecutionTagger(Session session, Metadata metadata, NodePartitioningManager nodePartitioningManager) - { - this.session = requireNonNull(session, "session is null"); - this.metadata = requireNonNull(metadata, "metadata is null"); - this.nodePartitioningManager = requireNonNull(nodePartitioningManager, "nodePartitioningManager is null"); - this.groupedExecutionEnabled = SystemSessionProperties.isGroupedExecutionEnabled(session); - } - - @Override - protected GroupedExecutionProperties visitPlan(PlanNode node, Void context) - { - if (node.getSources().isEmpty()) { - return GroupedExecutionProperties.notCapable(); - } - return processChildren(node); - } - - @Override - public GroupedExecutionProperties visitJoin(JoinNode node, Void context) - { - GroupedExecutionProperties left = node.getLeft().accept(this, null); - GroupedExecutionProperties right = node.getRight().accept(this, null); - - if (!groupedExecutionEnabled) { - return GroupedExecutionProperties.notCapable(); - } - - if (node.getDistributionType().isEmpty()) { - // This is possible when the optimizers is invoked with `forceSingleNode` set to true. - return GroupedExecutionProperties.notCapable(); - } - - if ((node.getType() == JoinNode.Type.RIGHT || node.getType() == JoinNode.Type.FULL) && !right.currentNodeCapable) { - // For a plan like this, if the fragment participates in grouped execution, - // the LookupOuterOperator corresponding to the RJoin will not work execute properly. - // - // * The operator has to execute as not-grouped because it can only look at the "used" flags in - // join build after all probe has finished. - // * The operator has to execute as grouped the subsequent LJoin expects that incoming - // operators are grouped. Otherwise, the LJoin won't be able to throw out the build side - // for each group as soon as the group completes. - // - // LJoin - // / \ - // RJoin Scan - // / \ - // Scan Remote - // - // TODO: - // The RJoin can still execute as grouped if there is no subsequent operator that depends - // on the RJoin being executed in a grouped manner. However, this is not currently implemented. - // Support for this scenario is already implemented in the execution side. - return GroupedExecutionProperties.notCapable(); - } - - switch (node.getDistributionType().get()) { - case REPLICATED: - // Broadcast join maintains partitioning for the left side. - // Right side of a broadcast is not capable of grouped execution because it always comes from a remote exchange. - checkState(!right.currentNodeCapable); - return left; - case PARTITIONED: - if (left.currentNodeCapable && right.currentNodeCapable) { - return new GroupedExecutionProperties( - true, - true, - ImmutableList.builder() - .addAll(left.capableTableScanNodes) - .addAll(right.capableTableScanNodes) - .build()); - } - // right.subTreeUseful && !left.currentNodeCapable: - // It's not particularly helpful to do grouped execution on the right side - // because the benefit is likely cancelled out due to required buffering for hash build. - // In theory, it could still be helpful (e.g. when the underlying aggregation's intermediate group state maybe larger than aggregation output). - // However, this is not currently implemented. JoinBridgeManager need to support such a lifecycle. - // !right.currentNodeCapable: - // The build/right side needs to buffer fully for this JOIN, but the probe/left side will still stream through. - // As a result, there is no reason to change currentNodeCapable or subTreeUseful to false. - // - return left; - } - throw new UnsupportedOperationException("Unknown distribution type: " + node.getDistributionType()); - } - - @Override - public GroupedExecutionProperties visitAggregation(AggregationNode node, Void context) - { - GroupedExecutionProperties properties = node.getSource().accept(this, null); - if (groupedExecutionEnabled && properties.isCurrentNodeCapable()) { - switch (node.getStep()) { - case SINGLE: - case FINAL: - return new GroupedExecutionProperties(true, true, properties.capableTableScanNodes); - case PARTIAL: - case INTERMEDIATE: - return properties; - } - } - return GroupedExecutionProperties.notCapable(); - } - - @Override - public GroupedExecutionProperties visitWindow(WindowNode node, Void context) - { - return processWindowFunction(node); - } - - @Override - public GroupedExecutionProperties visitRowNumber(RowNumberNode node, Void context) - { - return processWindowFunction(node); - } - - @Override - public GroupedExecutionProperties visitTopNRanking(TopNRankingNode node, Void context) - { - return processWindowFunction(node); - } - - private GroupedExecutionProperties processWindowFunction(PlanNode node) - { - GroupedExecutionProperties properties = getOnlyElement(node.getSources()).accept(this, null); - if (groupedExecutionEnabled && properties.isCurrentNodeCapable()) { - return new GroupedExecutionProperties(true, true, properties.capableTableScanNodes); - } - return GroupedExecutionProperties.notCapable(); - } - - @Override - public GroupedExecutionProperties visitPatternRecognition(PatternRecognitionNode node, Void context) - { - return GroupedExecutionProperties.notCapable(); - } - - @Override - public GroupedExecutionProperties visitTableScan(TableScanNode node, Void context) - { - Optional tablePartitioning = metadata.getTableProperties(session, node.getTable()).getTablePartitioning(); - if (tablePartitioning.isEmpty() || !node.isUseConnectorNodePartitioning()) { - return GroupedExecutionProperties.notCapable(); - } - List partitionHandles = nodePartitioningManager.listPartitionHandles(session, tablePartitioning.get().getPartitioningHandle()); - if (ImmutableList.of(NOT_PARTITIONED).equals(partitionHandles)) { - return new GroupedExecutionProperties(false, false, ImmutableList.of()); - } - return new GroupedExecutionProperties(true, false, ImmutableList.of(node.getId())); - } - - private GroupedExecutionProperties processChildren(PlanNode node) - { - // Each fragment has a partitioning handle, which is derived from leaf nodes in the fragment. - // Leaf nodes with different partitioning handle are not allowed to share a single fragment - // (except for special cases as detailed in addSourceDistribution). - // As a result, it is not necessary to check the compatibility between node.getSources because - // they are guaranteed to be compatible. - - // * If any child is "not capable", return "not capable" - // * When all children are capable ("capable and useful" or "capable but not useful") - // * if any child is "capable and useful", return "capable and useful" - // * if no children is "capable and useful", return "capable but not useful" - boolean anyUseful = false; - ImmutableList.Builder capableTableScanNodes = ImmutableList.builder(); - for (PlanNode source : node.getSources()) { - GroupedExecutionProperties properties = source.accept(this, null); - if (!properties.isCurrentNodeCapable()) { - return GroupedExecutionProperties.notCapable(); - } - anyUseful |= properties.isSubTreeUseful(); - capableTableScanNodes.addAll(properties.capableTableScanNodes); - } - return new GroupedExecutionProperties(true, anyUseful, capableTableScanNodes.build()); - } - } - - private static class GroupedExecutionProperties - { - // currentNodeCapable: - // Whether grouped execution is possible with the current node. - // For example, a table scan is capable iff it supports addressable split discovery. - // subTreeUseful: - // Whether grouped execution is beneficial in the current node, or any node below it. - // For example, a JOIN can benefit from grouped execution because build can be flushed early, reducing peak memory requirement. - // - // In the current implementation, subTreeUseful implies currentNodeCapable. - // In theory, this doesn't have to be the case. Take an example where a GROUP BY feeds into the build side of a JOIN. - // Even if JOIN cannot take advantage of grouped execution, it could still be beneficial to execute the GROUP BY with grouped execution - // (e.g. when the underlying aggregation's intermediate group state may be larger than aggregation output). - - private final boolean currentNodeCapable; - private final boolean subTreeUseful; - private final List capableTableScanNodes; - - public GroupedExecutionProperties(boolean currentNodeCapable, boolean subTreeUseful, List capableTableScanNodes) - { - this.currentNodeCapable = currentNodeCapable; - this.subTreeUseful = subTreeUseful; - this.capableTableScanNodes = ImmutableList.copyOf(requireNonNull(capableTableScanNodes, "capableTableScanNodes is null")); - // Verify that `subTreeUseful` implies `currentNodeCapable` - checkArgument(!subTreeUseful || currentNodeCapable); - checkArgument(currentNodeCapable == !capableTableScanNodes.isEmpty()); - } - - public static GroupedExecutionProperties notCapable() - { - return new GroupedExecutionProperties(false, false, ImmutableList.of()); - } - - public boolean isCurrentNodeCapable() - { - return currentNodeCapable; - } - - public boolean isSubTreeUseful() - { - return subTreeUseful; - } - - public List getCapableTableScanNodes() - { - return capableTableScanNodes; - } - } - private static final class PartitioningHandleReassigner extends SimplePlanRewriter { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/SplitSourceFactory.java b/core/trino-main/src/main/java/io/trino/sql/planner/SplitSourceFactory.java index 954f85608f83..d999d00d3bd3 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/SplitSourceFactory.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/SplitSourceFactory.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableMap; import io.airlift.log.Logger; import io.trino.Session; -import io.trino.operator.StageExecutionDescriptor; import io.trino.server.DynamicFilterService; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.DynamicFilter; @@ -75,8 +74,6 @@ import java.util.Optional; import static com.google.common.collect.Iterables.getOnlyElement; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.connector.DynamicFilter.EMPTY; import static io.trino.sql.ExpressionUtils.filterConjuncts; @@ -106,7 +103,7 @@ public Map createSplitSources(Session session, PlanFrag try { // get splits for this fragment, this is lazy so split assignments aren't actually calculated here return fragment.getRoot().accept( - new Visitor(session, fragment.getStageExecutionDescriptor(), TypeProvider.copyOf(fragment.getSymbols()), allSplitSources), + new Visitor(session, TypeProvider.copyOf(fragment.getSymbols()), allSplitSources), null); } catch (Throwable t) { @@ -129,18 +126,15 @@ private final class Visitor extends PlanVisitor, Void> { private final Session session; - private final StageExecutionDescriptor stageExecutionDescriptor; private final TypeProvider typeProvider; private final ImmutableList.Builder splitSources; private Visitor( Session session, - StageExecutionDescriptor stageExecutionDescriptor, TypeProvider typeProvider, ImmutableList.Builder allSplitSources) { this.session = session; - this.stageExecutionDescriptor = stageExecutionDescriptor; this.typeProvider = typeProvider; this.splitSources = allSplitSources; } @@ -183,7 +177,6 @@ private Map visitScanAndFilter(TableScanNode node, Opti SplitSource splitSource = splitManager.getSplits( session, node.getTable(), - stageExecutionDescriptor.isScanGroupedExecution(node.getId()) ? GROUPED_SCHEDULING : UNGROUPED_SCHEDULING, dynamicFilter, constraint); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java index 88bd7043475d..a8c44e663fb0 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/ExtractSpatialJoins.java @@ -20,7 +20,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.geospatial.KdbTree; import io.trino.geospatial.KdbTreeUtils; import io.trino.matching.Capture; @@ -81,10 +80,8 @@ import static io.trino.SystemSessionProperties.isSpatialJoinEnabled; import static io.trino.matching.Capture.newCapture; import static io.trino.spi.StandardErrorCode.INVALID_SPATIAL_PARTITIONING; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.connector.DynamicFilter.EMPTY; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -470,9 +467,9 @@ private static KdbTree loadKdbTree(String tableName, Session session, Metadata m ColumnHandle kdbTreeColumn = Iterables.getOnlyElement(visibleColumnHandles); Optional kdbTree = Optional.empty(); - try (SplitSource splitSource = splitManager.getSplits(session, tableHandle, UNGROUPED_SCHEDULING, EMPTY, alwaysTrue())) { + try (SplitSource splitSource = splitManager.getSplits(session, tableHandle, EMPTY, alwaysTrue())) { while (!Thread.currentThread().isInterrupted()) { - SplitBatch splitBatch = getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), 1000)); + SplitBatch splitBatch = getFutureValue(splitSource.getNextBatch(1000)); List splits = splitBatch.getSplits(); for (Split split : splits) { diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java index fe2a09f740b2..9463157fdacb 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/planprinter/PlanPrinter.java @@ -33,7 +33,6 @@ import io.trino.metadata.FunctionManager; import io.trino.metadata.Metadata; import io.trino.metadata.TableHandle; -import io.trino.operator.StageExecutionDescriptor; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.NullableValue; @@ -143,9 +142,6 @@ import static com.google.common.collect.ImmutableMap.toImmutableMap; import static io.trino.execution.StageInfo.getAllStages; import static io.trino.metadata.ResolvedFunction.extractFunctionName; -import static io.trino.operator.StageExecutionDescriptor.StageExecutionStrategy; -import static io.trino.operator.StageExecutionDescriptor.StageExecutionStrategy.UNGROUPED_EXECUTION; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.server.DynamicFilterService.DynamicFilterDomainStats; import static io.trino.sql.DynamicFilters.extractDynamicFilters; import static io.trino.sql.ExpressionUtils.combineConjunctsWithDuplicates; @@ -178,7 +174,6 @@ public class PlanPrinter PlanPrinter( PlanNode planRoot, TypeProvider types, - Optional stageExecutionStrategy, Function tableInfoSupplier, Map dynamicFilterDomainStats, ValuePrinter valuePrinter, @@ -211,7 +206,7 @@ public class PlanPrinter this.representation = new PlanRepresentation(planRoot, types, totalCpuTime, totalScheduledTime, totalBlockedTime); - Visitor visitor = new Visitor(stageExecutionStrategy, types, estimatedStatsAndCosts, stats); + Visitor visitor = new Visitor(types, estimatedStatsAndCosts, stats); planRoot.accept(visitor, null); } @@ -239,7 +234,7 @@ public static String jsonFragmentPlan(PlanNode root, Map symbols, TableInfoSupplier tableInfoSupplier = new TableInfoSupplier(metadata, session); ValuePrinter valuePrinter = new ValuePrinter(metadata, functionManager, session); - return new PlanPrinter(root, typeProvider, Optional.empty(), tableInfoSupplier, ImmutableMap.of(), valuePrinter, StatsAndCosts.empty(), Optional.empty()).toJson(); + return new PlanPrinter(root, typeProvider, tableInfoSupplier, ImmutableMap.of(), valuePrinter, StatsAndCosts.empty(), Optional.empty()).toJson(); } public static String jsonLogicalPlan( @@ -255,7 +250,6 @@ public static String jsonLogicalPlan( return new PlanPrinter( plan, types, - Optional.empty(), tableInfoSupplier, ImmutableMap.of(), valuePrinter, @@ -275,7 +269,7 @@ public static String textLogicalPlan( { TableInfoSupplier tableInfoSupplier = new TableInfoSupplier(metadata, session); ValuePrinter valuePrinter = new ValuePrinter(metadata, functionManager, session); - return new PlanPrinter(plan, types, Optional.empty(), tableInfoSupplier, ImmutableMap.of(), valuePrinter, estimatedStatsAndCosts, Optional.empty()).toText(verbose, level); + return new PlanPrinter(plan, types, tableInfoSupplier, ImmutableMap.of(), valuePrinter, estimatedStatsAndCosts, Optional.empty()).toText(verbose, level); } public static String textDistributedPlan( @@ -411,13 +405,11 @@ private static String formatFragment( Joiner.on(", ").join(arguments), formatHash(partitioningScheme.getHashColumn()))); } - builder.append(indentString(1)).append(format("Stage Execution Strategy: %s\n", fragment.getStageExecutionDescriptor().getStageExecutionStrategy())); builder.append( new PlanPrinter( fragment.getRoot(), typeProvider, - Optional.of(fragment.getStageExecutionDescriptor()), tableInfoSupplier, dynamicFilterDomainStats, valuePrinter, @@ -446,7 +438,6 @@ public static String graphvizLogicalPlan(PlanNode plan, TypeProvider types) SINGLE_DISTRIBUTION, ImmutableList.of(plan.getId()), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), plan.getOutputSymbols()), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); return GraphvizPrinter.printLogical(ImmutableList.of(fragment)); @@ -460,14 +451,12 @@ public static String graphvizDistributedPlan(SubPlan plan) private class Visitor extends PlanVisitor { - private final Optional stageExecutionStrategy; private final TypeProvider types; private final StatsAndCosts estimatedStatsAndCosts; private final Optional> stats; - public Visitor(Optional stageExecutionStrategy, TypeProvider types, StatsAndCosts estimatedStatsAndCosts, Optional> stats) + public Visitor(TypeProvider types, StatsAndCosts estimatedStatsAndCosts, Optional> stats) { - this.stageExecutionStrategy = requireNonNull(stageExecutionStrategy, "stageExecutionStrategy is null"); this.types = requireNonNull(types, "types is null"); this.estimatedStatsAndCosts = requireNonNull(estimatedStatsAndCosts, "estimatedStatsAndCosts is null"); this.stats = requireNonNull(stats, "stats is null"); @@ -975,19 +964,7 @@ public Void visitTableScan(TableScanNode node, Void context) NodeRepresentation nodeOutput; ImmutableMap.Builder descriptor = ImmutableMap.builder(); descriptor.put("table", table.toString()); - if (stageExecutionStrategy.isPresent()) { - StageExecutionStrategy executionStrategy = stageExecutionStrategy.get().isScanGroupedExecution(node.getId()) - ? stageExecutionStrategy.get().getStageExecutionStrategy() - : UNGROUPED_EXECUTION; - - nodeOutput = addNode( - node, - "TableScan", - descriptor.put("stageExecutionStrategy", executionStrategy.name()).buildOrThrow()); - } - else { - nodeOutput = addNode(node, "TableScan", descriptor.buildOrThrow()); - } + nodeOutput = addNode(node, "TableScan", descriptor.buildOrThrow()); printTableScanInfo(nodeOutput, node); return null; } @@ -1065,12 +1042,6 @@ private Void visitScanFilterAndProjectInfo( if (scanNode.isPresent()) { operatorName += "Scan"; descriptor.put("table", scanNode.get().getTable().toString()); - stageExecutionStrategy.ifPresent(executionDescriptor -> { - StageExecutionStrategy executionStrategy = executionDescriptor.isScanGroupedExecution(node.getId()) - ? executionDescriptor.getStageExecutionStrategy() - : UNGROUPED_EXECUTION; - descriptor.put("stageExecutionStrategy", executionStrategy.name()); - }); } List dynamicFilters = ImmutableList.of(); diff --git a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java index c14eab3aa9ba..7a45c14a20ed 100644 --- a/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java +++ b/core/trino-main/src/main/java/io/trino/testing/LocalQueryRunner.java @@ -55,7 +55,6 @@ import io.trino.exchange.ExchangeManagerRegistry; import io.trino.execution.DynamicFilterConfig; import io.trino.execution.FailureInjector.InjectedFailureType; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap; import io.trino.execution.QueryManagerConfig; import io.trino.execution.QueryPreparer; @@ -112,7 +111,6 @@ import io.trino.operator.OutputFactory; import io.trino.operator.PagesIndex; import io.trino.operator.PagesIndexPageSorter; -import io.trino.operator.StageExecutionDescriptor; import io.trino.operator.TaskContext; import io.trino.operator.TrinoOperatorFactories; import io.trino.operator.index.IndexJoinLookupStats; @@ -233,11 +231,8 @@ import static io.trino.connector.CatalogServiceProviderModule.createTableProceduresPropertyManager; import static io.trino.connector.CatalogServiceProviderModule.createTableProceduresProvider; import static io.trino.connector.CatalogServiceProviderModule.createTablePropertyManager; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.connector.DynamicFilter.EMPTY; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.sql.ParameterUtils.parameterExtractor; import static io.trino.sql.ParsingUtil.createParsingOptions; import static io.trino.sql.planner.LogicalPlanner.Stage.OPTIMIZED_AND_VALIDATED; @@ -446,7 +441,7 @@ private LocalQueryRunner( this.costCalculator = new CostCalculatorUsingExchanges(taskCountEstimator); this.estimatedExchangesCostCalculator = new CostCalculatorWithEstimatedExchanges(costCalculator, taskCountEstimator); - this.planFragmenter = new PlanFragmenter(metadata, functionManager, nodePartitioningManager, new QueryManagerConfig()); + this.planFragmenter = new PlanFragmenter(metadata, functionManager, new QueryManagerConfig()); GlobalSystemConnectorFactory globalSystemConnectorFactory = new GlobalSystemConnectorFactory(ImmutableSet.of( new NodeSystemTable(nodeManager), @@ -955,10 +950,8 @@ private List createDrivers(Session session, Plan plan, OutputFactory out exchangeManagerRegistry); // plan query - StageExecutionDescriptor stageExecutionDescriptor = subplan.getFragment().getStageExecutionDescriptor(); LocalExecutionPlan localExecutionPlan = executionPlanner.plan( taskContext, - stageExecutionDescriptor, subplan.getFragment().getRoot(), subplan.getFragment().getPartitioningScheme().getOutputLayout(), plan.getTypes(), @@ -974,7 +967,6 @@ private List createDrivers(Session session, Plan plan, OutputFactory out SplitSource splitSource = splitManager.getSplits( session, table, - stageExecutionDescriptor.isScanGroupedExecution(tableScan.getId()) ? GROUPED_SCHEDULING : UNGROUPED_SCHEDULING, EMPTY, alwaysTrue()); @@ -1133,7 +1125,7 @@ private AnalyzerFactory createAnalyzerFactory(QueryExplainerFactory queryExplain private static List getNextBatch(SplitSource splitSource) { - return getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), 1000)).getSplits(); + return getFutureValue(splitSource.getNextBatch(1000)).getSplits(); } private static List findTableScanNodes(PlanNode node) diff --git a/core/trino-main/src/main/java/io/trino/testing/TestingSplitManager.java b/core/trino-main/src/main/java/io/trino/testing/TestingSplitManager.java index 29c5573e3228..aff6d6ccb861 100644 --- a/core/trino-main/src/main/java/io/trino/testing/TestingSplitManager.java +++ b/core/trino-main/src/main/java/io/trino/testing/TestingSplitManager.java @@ -43,7 +43,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java index 5eb3236167f0..df5f4efb72da 100644 --- a/core/trino-main/src/test/java/io/trino/connector/MockConnector.java +++ b/core/trino-main/src/test/java/io/trino/connector/MockConnector.java @@ -251,7 +251,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java index bcbaf76c8c75..5333b933c669 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestCostCalculator.java @@ -111,7 +111,6 @@ public void setUp() planFragmenter = new PlanFragmenter( localQueryRunner.getMetadata(), localQueryRunner.getFunctionManager(), - localQueryRunner.getNodePartitioningManager(), new QueryManagerConfig()); } diff --git a/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java b/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java index 928b4e42f346..f167a1b256d7 100644 --- a/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/BenchmarkNodeScheduler.java @@ -156,7 +156,7 @@ public void setup() InternalNode node = nodes.get(i); ImmutableList.Builder initialSplits = ImmutableList.builder(); for (int j = 0; j < MAX_SPLITS_PER_NODE + MAX_PENDING_SPLITS_PER_TASK_PER_NODE; j++) { - initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(i), Lifespan.taskWide())); + initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(i))); } TaskId taskId = new TaskId(new StageId("test", 1), i, 0); MockRemoteTaskFactory.MockRemoteTask remoteTask = remoteTaskFactory.createTableScanTask(taskId, node, initialSplits.build(), nodeTaskMap.createPartitionedSplitCountTracker(node, taskId)); @@ -165,7 +165,7 @@ public void setup() } for (int i = 0; i < SPLITS; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(ThreadLocalRandom.current().nextInt(DATA_NODES)), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(ThreadLocalRandom.current().nextInt(DATA_NODES)))); } InMemoryNodeManager nodeManager = new InMemoryNodeManager(); diff --git a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java index 2bf03012d9e2..8a8a6f35746a 100644 --- a/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/MockRemoteTaskFactory.java @@ -81,7 +81,6 @@ import static io.trino.execution.buffer.OutputBuffers.BufferType.BROADCAST; import static io.trino.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; @@ -121,7 +120,6 @@ public MockRemoteTask createTableScanTask(TaskId taskId, InternalNode newNode, L SOURCE_DISTRIBUTION, ImmutableList.of(sourceId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol)), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); @@ -254,7 +252,6 @@ public TaskInfo getTaskInfo() state, location, nodeId, - ImmutableSet.of(), failures, 0, 0, @@ -288,7 +285,6 @@ public TaskStatus getTaskStatus() taskStateMachine.getState(), location, nodeId, - ImmutableSet.of(), ImmutableList.of(), queuedSplitsInfo.getCount(), combinedSplitsInfo.getCount() - queuedSplitsInfo.getCount(), @@ -401,12 +397,6 @@ public synchronized void noMoreSplits(PlanNodeId sourceId) } } - @Override - public void noMoreSplits(PlanNodeId sourceId, Lifespan lifespan) - { - throw new UnsupportedOperationException(); - } - @Override public void setOutputBuffers(OutputBuffers outputBuffers) { 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 e5d917c8b28d..5bc9d1ffeda1 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 @@ -61,7 +61,6 @@ import java.util.Optional; import static io.trino.SessionTestUtils.TEST_SESSION; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; @@ -78,7 +77,7 @@ private TaskTestUtils() {} private static final CatalogName CONNECTOR_ID = TEST_TABLE_HANDLE.getCatalogName(); - public static final ScheduledSplit SPLIT = new ScheduledSplit(0, TABLE_SCAN_NODE_ID, new Split(CONNECTOR_ID, TestingSplit.createLocalSplit(), Lifespan.taskWide())); + public static final ScheduledSplit SPLIT = new ScheduledSplit(0, TABLE_SCAN_NODE_ID, new Split(CONNECTOR_ID, TestingSplit.createLocalSplit())); public static final ImmutableList EMPTY_SPLIT_ASSIGNMENTS = ImmutableList.of(); @@ -98,7 +97,6 @@ private TaskTestUtils() {} ImmutableList.of(TABLE_SCAN_NODE_ID), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(SYMBOL)) .withBucketToPartition(Optional.of(new int[1])), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java b/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java index 800cab175dd1..4be21eab512b 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestNodeScheduler.java @@ -153,7 +153,7 @@ public void tearDown() public void testAssignmentWhenNoNodes() { Set splits = new HashSet<>(); - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); assertTrinoExceptionThrownBy(() -> nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values()))) .hasErrorCode(NO_NODES_AVAILABLE) @@ -164,7 +164,7 @@ public void testAssignmentWhenNoNodes() public void testScheduleLocal() { setUpNodes(); - Split split = new Split(CONNECTOR_ID, new TestSplitLocallyAccessible(), Lifespan.taskWide()); + Split split = new Split(CONNECTOR_ID, new TestSplitLocallyAccessible()); Set splits = ImmutableSet.of(split); Map.Entry assignment = getOnlyElement(nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments().entries()); @@ -200,7 +200,7 @@ public void testTopologyAwareScheduling() // Fill up the nodes with non-local data ImmutableSet.Builder nonRackLocalBuilder = ImmutableSet.builder(); for (int i = 0; i < (25 + 11) * 3; i++) { - nonRackLocalBuilder.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("data.other_rack", 1)), Lifespan.taskWide())); + nonRackLocalBuilder.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("data.other_rack", 1)))); } Set nonRackLocalSplits = nonRackLocalBuilder.build(); Multimap assignments = nodeSelector.computeAssignments(nonRackLocalSplits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -232,10 +232,10 @@ public void testTopologyAwareScheduling() HostAddress dataHost1 = HostAddress.fromParts("data.rack1", 1); HostAddress dataHost2 = HostAddress.fromParts("data.rack2", 1); for (int i = 0; i < 6 * 2; i++) { - rackLocalSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(dataHost1), Lifespan.taskWide())); + rackLocalSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(dataHost1))); } for (int i = 0; i < 6; i++) { - rackLocalSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(dataHost2), Lifespan.taskWide())); + rackLocalSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(dataHost2))); } assignments = nodeSelector.computeAssignments(rackLocalSplits.build(), ImmutableList.copyOf(taskMap.values())).getAssignments(); for (InternalNode node : assignments.keySet()) { @@ -276,9 +276,9 @@ public void testTopologyAwareScheduling() // Assign local splits ImmutableSet.Builder localSplits = ImmutableSet.builder(); - localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host1.rack1", 1)), Lifespan.taskWide())); - localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host2.rack1", 1)), Lifespan.taskWide())); - localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host3.rack2", 1)), Lifespan.taskWide())); + localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host1.rack1", 1)))); + localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host2.rack1", 1)))); + localSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromParts("host3.rack2", 1)))); assignments = nodeSelector.computeAssignments(localSplits.build(), ImmutableList.copyOf(taskMap.values())).getAssignments(); assertEquals(assignments.size(), 3); assertEquals(assignments.keySet().size(), 3); @@ -289,7 +289,7 @@ public void testScheduleRemote() { setUpNodes(); Set splits = new HashSet<>(); - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); Multimap assignments = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); assertEquals(assignments.size(), 1); } @@ -301,7 +301,7 @@ public void testBasicAssignment() // One split for each node Set splits = new HashSet<>(); for (int i = 0; i < 3; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } Multimap assignments = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); assertEquals(assignments.entries().size(), 3); @@ -319,7 +319,7 @@ public void testMaxSplitsPerNode() ImmutableList.Builder initialSplits = ImmutableList.builder(); for (int i = 0; i < 10; i++) { - initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } MockRemoteTaskFactory remoteTaskFactory = new MockRemoteTaskFactory(remoteTaskExecutor, remoteTaskScheduledExecutor); @@ -334,7 +334,7 @@ public void testMaxSplitsPerNode() Set splits = new HashSet<>(); for (int i = 0; i < 5; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } Multimap assignments = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -358,7 +358,7 @@ public void testBasicAssignmentMaxUnacknowledgedSplitsPerTask() int splitCount = nodeCount + 1; Set splits = new HashSet<>(); for (int i = 0; i < splitCount; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } Multimap assignments = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); assertEquals(assignments.entries().size(), nodeCount); @@ -376,7 +376,7 @@ public void testMaxSplitsPerNodePerTask() ImmutableList.Builder initialSplits = ImmutableList.builder(); for (int i = 0; i < 20; i++) { - initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } List tasks = new ArrayList<>(); @@ -398,7 +398,7 @@ public void testMaxSplitsPerNodePerTask() Set splits = new HashSet<>(); for (int i = 0; i < 5; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } Multimap assignments = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -424,7 +424,7 @@ public void testTaskCompletion() RemoteTask remoteTask = remoteTaskFactory.createTableScanTask( taskId, chosenNode, - ImmutableList.of(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())), + ImmutableList.of(new Split(CONNECTOR_ID, new TestSplitRemote())), nodeTaskMap.createPartitionedSplitCountTracker(chosenNode, taskId)); nodeTaskMap.addTask(chosenNode, remoteTask); assertEquals(nodeTaskMap.getPartitionedSplitsOnNode(chosenNode).getCount(), 1); @@ -447,15 +447,15 @@ public void testSplitCount() RemoteTask remoteTask1 = remoteTaskFactory.createTableScanTask(taskId1, chosenNode, ImmutableList.of( - new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide()), - new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())), + new Split(CONNECTOR_ID, new TestSplitRemote()), + new Split(CONNECTOR_ID, new TestSplitRemote())), nodeTaskMap.createPartitionedSplitCountTracker(chosenNode, taskId1)); TaskId taskId2 = new TaskId(new StageId("test", 1), 2, 0); RemoteTask remoteTask2 = remoteTaskFactory.createTableScanTask( taskId2, chosenNode, - ImmutableList.of(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())), + ImmutableList.of(new Split(CONNECTOR_ID, new TestSplitRemote())), nodeTaskMap.createPartitionedSplitCountTracker(chosenNode, taskId2)); nodeTaskMap.addTask(chosenNode, remoteTask1); @@ -478,7 +478,7 @@ public void testPrioritizedAssignmentOfLocalSplit() Set splits = new LinkedHashSet<>(); // 20 splits with node1 as a non-local node to be assigned in the second iteration of computeAssignments for (int i = 0; i < 20; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } // computeAssignments just returns a mapping of nodes with splits to be assigned, it does not assign splits Multimap initialAssignment = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -489,9 +489,9 @@ public void testPrioritizedAssignmentOfLocalSplit() // Check for assignment of splits beyond maxSplitsPerNode (2 splits should remain unassigned) // 1 split with node1 as local node - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); // 1 split with node1 as a non-local node - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); //splits now contains 22 splits : 1 with node1 as local node and 21 with node1 as a non-local node Multimap finalAssignment = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); // Check that only 20 splits are being assigned as there is a single task @@ -517,11 +517,11 @@ public void testAssignmentWhenMixedSplits() Set splits = new LinkedHashSet<>(); // 10 splits with node1 as local node to be assigned in the first iteration of computeAssignments for (int i = 0; i < 10; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); } // 10 splits with node1 as a non-local node to be assigned in the second iteration of computeAssignments for (int i = 0; i < 10; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } // computeAssignments just returns a mapping of nodes with splits to be assigned, it does not assign splits Multimap initialAssignment = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -532,9 +532,9 @@ public void testAssignmentWhenMixedSplits() // Check for assignment of splits beyond maxSplitsPerNode (2 splits should remain unassigned) // 1 split with node1 as local node - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); // 1 split with node1 as a non-local node - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); //splits now contains 22 splits : 11 with node1 as local node and 11 with node1 as a non-local node Multimap finalAssignment = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); // Check that only 20 splits are being assigned as there is a single task @@ -561,7 +561,7 @@ public void testOptimizedLocalScheduling() Set splits = new LinkedHashSet<>(); // 20 splits with node1 as local node to be assigned in the first iteration of computeAssignments for (int i = 0; i < 20; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); } // computeAssignments just returns a mapping of nodes with splits to be assigned, it does not assign splits Multimap assignments1 = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -573,7 +573,7 @@ public void testOptimizedLocalScheduling() // 19 splits with node2 as local node to be assigned in the first iteration of computeAssignments for (int i = 0; i < 19; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromString("10.0.0.1:12")), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromString("10.0.0.1:12")))); } Multimap assignments2 = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); // Check that all 39 splits are being assigned (20 splits assigned to node1 and 19 splits assigned to node2) @@ -595,9 +595,9 @@ public void testOptimizedLocalScheduling() assertEquals(node2Splits, 19); // 1 split with node1 as local node - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); // 1 split with node2 as local node - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromString("10.0.0.1:12")), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(HostAddress.fromString("10.0.0.1:12")))); //splits now contains 41 splits : 21 with node1 as local node and 20 with node2 as local node Multimap assignments3 = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); // Check that only 40 splits are being assigned as there is a single task @@ -637,7 +637,7 @@ public void testEquateDistribution() Set splits = new LinkedHashSet<>(); // 20 splits with node1 as local node to be assigned in the first iteration of computeAssignments for (int i = 0; i < 20; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); } // check that splits are divided uniformly across all nodes Multimap assignment = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -684,7 +684,7 @@ public void testEquateDistributionConsistentHashing(int numberOfNodes, int numbe // assign splits randomly according to consistent hashing for (int i = 0; i < numberOfSplits; i++) { InternalNode node = nodes.get(Hashing.consistentHash(random.nextInt(), nodes.size())); - Split split = new Split(CONNECTOR_ID, new TestSplitLocal(node.getHostAndPort()), Lifespan.taskWide()); + Split split = new Split(CONNECTOR_ID, new TestSplitLocal(node.getHostAndPort())); splits.add(split); originalAssignmentBuilder.put(node, split); } @@ -716,8 +716,8 @@ public void testRedistributeSplit() Set splitsAssignedToNode1 = new LinkedHashSet<>(); // Node1 to be assigned 12 splits out of which 6 are local to it for (int i = 0; i < 6; i++) { - splitsAssignedToNode1.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); - splitsAssignedToNode1.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splitsAssignedToNode1.add(new Split(CONNECTOR_ID, new TestSplitLocal())); + splitsAssignedToNode1.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } for (Split split : splitsAssignedToNode1) { assignment.put(node1, split); @@ -726,7 +726,7 @@ public void testRedistributeSplit() Set splitsAssignedToNode2 = new LinkedHashSet<>(); // Node2 to be assigned 10 splits for (int i = 0; i < 10; i++) { - splitsAssignedToNode2.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splitsAssignedToNode2.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } for (Split split : splitsAssignedToNode2) { assignment.put(node2, split); @@ -768,7 +768,7 @@ public void testEmptyAssignmentWithFullNodes() Set splits = new LinkedHashSet<>(); // 20 splits with node1 as local node to be assigned in the first iteration of computeAssignments for (int i = 0; i < (20 + 10 + 5) * 2; i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitLocal(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitLocal())); } // computeAssignments just returns a mapping of nodes with splits to be assigned, it does not assign splits Multimap assignments1 = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(taskMap.values())).getAssignments(); @@ -811,7 +811,7 @@ public void testMaxUnacknowledgedSplitsPerTask() setUpNodes(); ImmutableList.Builder initialSplits = ImmutableList.builder(); for (int i = 0; i < maxUnacknowledgedSplitsPerTask; i++) { - initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + initialSplits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } List nodes = new ArrayList<>(); @@ -833,7 +833,7 @@ public void testMaxUnacknowledgedSplitsPerTask() // One split per node Set splits = new HashSet<>(); for (int i = 0; i < nodes.size(); i++) { - splits.add(new Split(CONNECTOR_ID, new TestSplitRemote(), Lifespan.taskWide())); + splits.add(new Split(CONNECTOR_ID, new TestSplitRemote())); } SplitPlacementResult splitPlacements = nodeSelector.computeAssignments(splits, ImmutableList.copyOf(tasks)); // No splits should have been placed, max unacknowledged was already reached diff --git a/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java b/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java index 9ad9d5a90d53..51b001d6dbfe 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestSqlStage.java @@ -50,7 +50,6 @@ import static io.trino.execution.SqlStage.createSqlStage; import static io.trino.execution.buffer.OutputBuffers.BufferType.ARBITRARY; import static io.trino.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; @@ -135,7 +134,6 @@ private void testFinalStageInfoInternal() Optional.empty(), createInitialEmptyOutputBuffers(ARBITRARY), ImmutableMultimap.of(), - ImmutableMultimap.of(), ImmutableSet.of(), Optional.empty()); latch.countDown(); @@ -184,7 +182,6 @@ private static PlanFragment createExchangePlanFragment() SOURCE_DISTRIBUTION, ImmutableList.of(planNode.getId()), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), planNode.getOutputSymbols()), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); } 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 d9402ccfa50c..85011910961a 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 @@ -18,7 +18,6 @@ import com.google.common.base.Ticker; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import io.airlift.slice.Slice; @@ -40,15 +39,10 @@ import io.trino.metadata.Split; import io.trino.operator.DriverContext; import io.trino.operator.DriverFactory; -import io.trino.operator.Operator; import io.trino.operator.OperatorContext; -import io.trino.operator.OperatorFactory; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.SourceOperator; import io.trino.operator.SourceOperatorFactory; -import io.trino.operator.StageExecutionDescriptor; import io.trino.operator.TaskContext; -import io.trino.operator.ValuesOperator.ValuesOperatorFactory; import io.trino.operator.output.TaskOutputOperator.TaskOutputOperatorFactory; import io.trino.spi.HostAddress; import io.trino.spi.Page; @@ -61,18 +55,11 @@ import io.trino.sql.planner.LocalExecutionPlanner.LocalExecutionPlan; import io.trino.sql.planner.plan.PlanNodeId; import org.openjdk.jol.info.ClassLayout; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.OptionalInt; -import java.util.Queue; -import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; @@ -80,16 +67,12 @@ import java.util.function.Function; import java.util.function.Supplier; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.concurrent.Threads.threadsNamed; import static io.airlift.units.DataSize.Unit.GIGABYTE; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.SessionTestUtils.TEST_SESSION; import static io.trino.block.BlockAssertions.createStringSequenceBlock; -import static io.trino.block.BlockAssertions.createStringsBlock; import static io.trino.execution.TaskState.FINISHED; import static io.trino.execution.TaskState.FLUSHING; import static io.trino.execution.TaskState.RUNNING; @@ -99,8 +82,6 @@ import static io.trino.execution.buffer.OutputBuffers.createInitialEmptyOutputBuffers; import static io.trino.execution.buffer.PagesSerde.getSerializedPagePositionCount; import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; -import static io.trino.operator.PipelineExecutionStrategy.GROUPED_EXECUTION; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.type.VarcharType.VARCHAR; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newScheduledThreadPool; @@ -117,14 +98,8 @@ public class TestSqlTaskExecution private static final Duration ASSERT_WAIT_TIMEOUT = new Duration(1, HOURS); public static final TaskId TASK_ID = new TaskId(new StageId("query", 0), 0, 0); - @DataProvider - public static Object[][] executionStrategies() - { - return new Object[][] {{UNGROUPED_EXECUTION}, {GROUPED_EXECUTION}}; - } - - @Test(dataProvider = "executionStrategies", timeOut = 20_000) - public void testSimple(PipelineExecutionStrategy executionStrategy) + @Test + public void testSimple() throws Exception { ScheduledExecutorService taskNotificationExecutor = newScheduledThreadPool(10, threadsNamed("task-notification-%s")); @@ -142,14 +117,11 @@ public void testSimple(PipelineExecutionStrategy executionStrategy) // // pipeline 0 ... pipeline id // partitioned ... partitioned/unpartitioned pipeline - // grouped ... execution strategy (in grouped test) - // ungrouped ... execution strategy (in ungrouped test) // // TaskOutput // | // Scan // - // See #testComplex for all the bahaviors that are tested. Not all of them apply here. TestingScanOperatorFactory testingScanOperatorFactory = new TestingScanOperatorFactory(0, TABLE_SCAN_NODE_ID, ImmutableList.of(VARCHAR)); TaskOutputOperatorFactory taskOutputOperatorFactory = new TaskOutputOperatorFactory( 1, @@ -163,260 +135,8 @@ public void testSimple(PipelineExecutionStrategy executionStrategy) true, true, ImmutableList.of(testingScanOperatorFactory, taskOutputOperatorFactory), - OptionalInt.empty(), - executionStrategy)), - ImmutableList.of(TABLE_SCAN_NODE_ID), - executionStrategy == GROUPED_EXECUTION ? StageExecutionDescriptor.fixedLifespanScheduleGroupedExecution(ImmutableList.of(TABLE_SCAN_NODE_ID)) : StageExecutionDescriptor.ungroupedExecution()); - TaskContext taskContext = newTestingTaskContext(taskNotificationExecutor, driverYieldExecutor, taskStateMachine); - SqlTaskExecution sqlTaskExecution = SqlTaskExecution.createSqlTaskExecution( - taskStateMachine, - taskContext, - outputBuffer, - localExecutionPlan, - taskExecutor, - taskNotificationExecutor, - createTestSplitMonitor()); - - // - // test body - assertEquals(taskStateMachine.getState(), RUNNING); - - switch (executionStrategy) { - case UNGROUPED_EXECUTION: - // add assignment for pipeline - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - TABLE_SCAN_NODE_ID, - ImmutableSet.of(newScheduledSplit(0, TABLE_SCAN_NODE_ID, Lifespan.taskWide(), 100000, 123)), - false))); - // assert that partial task result is produced - outputBufferConsumer.consume(123, ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - // * operatorFactory will be closed even though operator can't execute - // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place - testingScanOperatorFactory.getPauser().pause(); - // add assignment for pipeline, mark as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - TABLE_SCAN_NODE_ID, - ImmutableSet.of( - newScheduledSplit(1, TABLE_SCAN_NODE_ID, Lifespan.taskWide(), 200000, 300), - newScheduledSplit(2, TABLE_SCAN_NODE_ID, Lifespan.taskWide(), 300000, 200)), - true))); - // assert that pipeline will have no more drivers - waitUntilEquals(testingScanOperatorFactory::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that no DriverGroup is fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of()); - // resume operator execution - testingScanOperatorFactory.getPauser().resume(); - // assert that task result is produced - outputBufferConsumer.consume(300 + 200, ASSERT_WAIT_TIMEOUT); - outputBufferConsumer.assertBufferComplete(ASSERT_WAIT_TIMEOUT); - - break; - case GROUPED_EXECUTION: - // add assignment for pipeline (driver group [1, 5]), mark driver group [1] as noMoreSplits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - TABLE_SCAN_NODE_ID, - ImmutableSet.of( - newScheduledSplit(0, TABLE_SCAN_NODE_ID, Lifespan.driverGroup(1), 0, 1), - newScheduledSplit(1, TABLE_SCAN_NODE_ID, Lifespan.driverGroup(5), 100000, 10)), - ImmutableSet.of(Lifespan.driverGroup(1)), - false))); - // assert that pipeline will have no more drivers for driver group [1] - waitUntilEquals(testingScanOperatorFactory::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(1)), ASSERT_WAIT_TIMEOUT); - // assert that partial result is produced for both driver groups - outputBufferConsumer.consume(1 + 10, ASSERT_WAIT_TIMEOUT); - // assert that driver group [1] is fully completed - waitUntilEquals(taskContext::getCompletedDriverGroups, ImmutableSet.of(Lifespan.driverGroup(1)), ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - // * operatorFactory will be closed even though operator can't execute - // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place - testingScanOperatorFactory.getPauser().pause(); - // add assignment for pipeline (driver group [5]), mark driver group [5] as noMoreSplits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - TABLE_SCAN_NODE_ID, - ImmutableSet.of(newScheduledSplit(2, TABLE_SCAN_NODE_ID, Lifespan.driverGroup(5), 200000, 300)), - ImmutableSet.of(Lifespan.driverGroup(5)), - false))); - // assert that pipeline will have no more drivers for driver group [1, 5] - waitUntilEquals(testingScanOperatorFactory::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(1), Lifespan.driverGroup(5)), ASSERT_WAIT_TIMEOUT); - // assert that driver group [5] is NOT YET fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of(Lifespan.driverGroup(1))); - // resume operator execution - testingScanOperatorFactory.getPauser().resume(); - // assert that partial result is produced - outputBufferConsumer.consume(300, ASSERT_WAIT_TIMEOUT); - // assert that driver group [1, 5] is fully completed - waitUntilEquals(taskContext::getCompletedDriverGroups, ImmutableSet.of(Lifespan.driverGroup(1), Lifespan.driverGroup(5)), ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - testingScanOperatorFactory.getPauser().pause(); - // add assignment for pipeline (driver group [7]), mark pipeline as noMoreSplits without explicitly marking driver group 7 - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - TABLE_SCAN_NODE_ID, - ImmutableSet.of( - newScheduledSplit(3, TABLE_SCAN_NODE_ID, Lifespan.driverGroup(7), 300000, 45), - newScheduledSplit(4, TABLE_SCAN_NODE_ID, Lifespan.driverGroup(7), 400000, 54)), - ImmutableSet.of(), - true))); - // assert that pipeline will have no more drivers for driver group [1, 5, 7] - waitUntilEquals(testingScanOperatorFactory::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(1), Lifespan.driverGroup(5), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - // assert that pipeline will have no more drivers - waitUntilEquals(testingScanOperatorFactory::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that driver group [1, 5] is fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of(Lifespan.driverGroup(1), Lifespan.driverGroup(5))); - // resume operator execution - testingScanOperatorFactory.getPauser().resume(); - // assert that result is produced - outputBufferConsumer.consume(45 + 54, ASSERT_WAIT_TIMEOUT); - outputBufferConsumer.assertBufferComplete(ASSERT_WAIT_TIMEOUT); - // assert that driver group [1, 5, 7] is fully completed - waitUntilEquals(taskContext::getCompletedDriverGroups, ImmutableSet.of(Lifespan.driverGroup(1), Lifespan.driverGroup(5), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - - break; - default: - throw new UnsupportedOperationException(); - } - - assertEquals(taskStateMachine.getStateChange(RUNNING).get(10, SECONDS), FLUSHING); - outputBufferConsumer.abort(); // complete the task by calling abort on it - assertEquals(taskStateMachine.getStateChange(FLUSHING).get(10, SECONDS), FINISHED); - } - finally { - taskExecutor.stop(); - taskNotificationExecutor.shutdownNow(); - driverYieldExecutor.shutdown(); - } - } - - @Test(dataProvider = "executionStrategies", timeOut = 20_000) - public void testComplex(PipelineExecutionStrategy executionStrategy) - throws Exception - { - 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.start(); - - try { - TaskStateMachine taskStateMachine = new TaskStateMachine(TASK_ID, taskNotificationExecutor); - PartitionedOutputBuffer outputBuffer = newTestingOutputBuffer(taskNotificationExecutor); - OutputBufferConsumer outputBufferConsumer = new OutputBufferConsumer(outputBuffer, OUTPUT_BUFFER_ID); - - // test initialization: complex test with 4 pipelines - // Take a task with the following set of pipelines for example: - // - // pipeline 0 pipeline 1 pipeline 2 pipeline 3 ... pipeline id - // partitioned unpartitioned partitioned unpartitioned ... partitioned/unpartitioned pipeline - // grouped grouped grouped ungrouped ... execution strategy (in grouped test) - // ungrouped ungrouped ungrouped ungrouped ... execution strategy (in ungrouped test) - // - // TaskOutput-0 - // | - // CrossJoin-C ................................... Build-C - // | | - // CrossJoin-A ..... Build-A Values-3 - // | | - // Scan-0 CrossJoin-B .... Build-B - // (effectively ExchangeSink) | - // | Scan-2 - // Values-1 - // (1 row) - // - // CrossJoin operator here has the same lifecycle behavior as a real cross/hash-join, and produces - // the correct number of rows, but doesn't actually produce a cross-join for simplicity. - // - // A single task can never have all 4 combinations: partitioned/unpartitioned x grouped/ungrouped. - // * In the case of ungrouped test, this test covers driver with - // 1) split lifecycle (partitioned ungrouped) - // 2) task lifecycle (unpartitioned ungrouped) - // These are the only 2 possible pipeline execution strategy a task can have if the task has ungrouped execution strategy. - // * In the case of grouped test, this covers: - // 1) split lifecycle (partitioned grouped) - // 2) driver group lifecycle (unpartitioned grouped) - // 3) task lifecycle (unpartitioned ungrouped) - // These are the only 3 possible pipeline execution strategy a task can have if the task has grouped execution strategy. - // - // The following behaviors are tested: - // * DriverFactory are marked as noMoreDriver/Operator for particular lifespans as soon as they can be: - // * immediately, if the pipeline has task lifecycle (ungrouped and unpartitioned). - // * when SplitAssignment containing the lifespan is encountered, if the pipeline has driver group lifecycle (grouped and unpartitioned). - // * when SplitAssignment indicate that no more splits will be produced for the plan node (and plan nodes that schedule before it - // due to phased scheduling) and lifespan combination, if the pipeline has split lifecycle (partitioned). - // * DriverFactory are marked as noMoreDriver/Operator as soon as they can be: - // * immediately, if the pipeline has task lifecycle (ungrouped and unpartitioned). - // * when SplitAssignment indicate that will no more splits, otherwise. - // * Driver groups are marked as completed as soon as they should be: - // * when there are no active driver, and all DriverFactory for the lifespan (across all pipelines) are marked as completed. - // * Rows are produced as soon as they should be: - // * streams data through as soon as the build side is ready, for CrossJoin - // * streams data through, otherwise. - PlanNodeId scan0NodeId = new PlanNodeId("scan-0"); - PlanNodeId values1NodeId = new PlanNodeId("values-1"); - PlanNodeId scan2NodeId = new PlanNodeId("scan-2"); - PlanNodeId values3NodeId = new PlanNodeId("values-3"); - PlanNodeId joinANodeId = new PlanNodeId("join-a"); - PlanNodeId joinBNodeId = new PlanNodeId("join-b"); - PlanNodeId joinCNodeId = new PlanNodeId("join-c"); - BuildStates buildStatesA = new BuildStates(executionStrategy); - BuildStates buildStatesB = new BuildStates(executionStrategy); - BuildStates buildStatesC = new BuildStates(UNGROUPED_EXECUTION); - TestingScanOperatorFactory scanOperatorFactory0 = new TestingScanOperatorFactory(1, scan0NodeId, ImmutableList.of(VARCHAR)); - ValuesOperatorFactory valuesOperatorFactory1 = new ValuesOperatorFactory( - 101, - values1NodeId, - ImmutableList.of(new Page(createStringsBlock("multiplier1")))); - TestingScanOperatorFactory scanOperatorFactory2 = new TestingScanOperatorFactory(201, scan2NodeId, ImmutableList.of(VARCHAR)); - ValuesOperatorFactory valuesOperatorFactory3 = new ValuesOperatorFactory( - 301, - values3NodeId, - ImmutableList.of(new Page(createStringsBlock("x", "y", "multiplier3")))); - TaskOutputOperatorFactory taskOutputOperatorFactory = new TaskOutputOperatorFactory( - 4, - joinCNodeId, - outputBuffer, - Function.identity(), - new PagesSerdeFactory(new TestingBlockEncodingSerde(), false)); - TestingCrossJoinOperatorFactory joinOperatorFactoryA = new TestingCrossJoinOperatorFactory(2, joinANodeId, buildStatesA); - TestingCrossJoinOperatorFactory joinOperatorFactoryB = new TestingCrossJoinOperatorFactory(102, joinBNodeId, buildStatesB); - TestingCrossJoinOperatorFactory joinOperatorFactoryC = new TestingCrossJoinOperatorFactory(3, joinCNodeId, buildStatesC); - TestingBuildOperatorFactory buildOperatorFactoryA = new TestingBuildOperatorFactory(103, joinANodeId, buildStatesA); - TestingBuildOperatorFactory buildOperatorFactoryB = new TestingBuildOperatorFactory(202, joinBNodeId, buildStatesB); - TestingBuildOperatorFactory buildOperatorFactoryC = new TestingBuildOperatorFactory(302, joinCNodeId, buildStatesC); - - LocalExecutionPlan localExecutionPlan = new LocalExecutionPlan( - ImmutableList.of( - new DriverFactory( - 0, - true, - true, - ImmutableList.of(scanOperatorFactory0, joinOperatorFactoryA, joinOperatorFactoryC, taskOutputOperatorFactory), - OptionalInt.empty(), - executionStrategy), - new DriverFactory( - 1, - false, - false, - ImmutableList.of(valuesOperatorFactory1, joinOperatorFactoryB, buildOperatorFactoryA), - OptionalInt.empty(), - executionStrategy), - new DriverFactory( - 2, - true, - false, - ImmutableList.of(scanOperatorFactory2, buildOperatorFactoryB), - OptionalInt.empty(), - executionStrategy), - new DriverFactory( - 3, - false, - false, - ImmutableList.of(valuesOperatorFactory3, buildOperatorFactoryC), - OptionalInt.empty(), - UNGROUPED_EXECUTION)), - ImmutableList.of(scan2NodeId, scan0NodeId), - executionStrategy == GROUPED_EXECUTION ? StageExecutionDescriptor.fixedLifespanScheduleGroupedExecution(ImmutableList.of(scan0NodeId, scan2NodeId)) : StageExecutionDescriptor.ungroupedExecution()); + OptionalInt.empty())), + ImmutableList.of(TABLE_SCAN_NODE_ID)); TaskContext taskContext = newTestingTaskContext(taskNotificationExecutor, driverYieldExecutor, taskStateMachine); SqlTaskExecution sqlTaskExecution = SqlTaskExecution.createSqlTaskExecution( taskStateMachine, @@ -431,154 +151,32 @@ public void testComplex(PipelineExecutionStrategy executionStrategy) // test body assertEquals(taskStateMachine.getState(), RUNNING); - switch (executionStrategy) { - case UNGROUPED_EXECUTION: - // assert that pipeline 1 and pipeline 3 will have no more drivers - // (Unpartitioned ungrouped pipelines can have all driver instance created up front.) - waitUntilEquals(joinOperatorFactoryB::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryA::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryC::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - - // add assignment for pipeline 2, and mark as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan2NodeId, - ImmutableSet.of( - newScheduledSplit(0, scan2NodeId, Lifespan.taskWide(), 100000, 1), - newScheduledSplit(1, scan2NodeId, Lifespan.taskWide(), 300000, 2)), - false))); - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan2NodeId, - ImmutableSet.of(newScheduledSplit(2, scan2NodeId, Lifespan.taskWide(), 300000, 2)), - true))); - // assert that pipeline 2 will have no more drivers - waitUntilEquals(scanOperatorFactory2::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryB::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - // * operatorFactory will be closed even though operator can't execute - // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place - scanOperatorFactory0.getPauser().pause(); - - // add assignment for pipeline 0, mark as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan0NodeId, - ImmutableSet.of(newScheduledSplit(3, scan0NodeId, Lifespan.taskWide(), 400000, 100)), - true))); - // assert that pipeline 0 will have no more drivers - waitUntilEquals(scanOperatorFactory0::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryA::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryC::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that no DriverGroup is fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of()); - // resume operator execution - scanOperatorFactory0.getPauser().resume(); - // assert that task result is produced - outputBufferConsumer.consume(100 * 5 * 3, ASSERT_WAIT_TIMEOUT); - outputBufferConsumer.assertBufferComplete(ASSERT_WAIT_TIMEOUT); - - break; - case GROUPED_EXECUTION: - // assert that pipeline 3 will have no more drivers - // (Unpartitioned ungrouped pipelines can have all driver instances created up front.) - waitUntilEquals(buildOperatorFactoryC::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - - // add assignment for pipeline 2 driver group 3, and mark driver group 3 as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan2NodeId, - ImmutableSet.of( - newScheduledSplit(0, scan2NodeId, Lifespan.driverGroup(3), 0, 1), - newScheduledSplit(1, scan2NodeId, Lifespan.driverGroup(3), 100000, 2)), - false))); - // assert that pipeline 1 driver group [3] will have no more drivers - waitUntilEquals(joinOperatorFactoryB::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryA::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan2NodeId, - ImmutableSet.of(newScheduledSplit(2, scan2NodeId, Lifespan.driverGroup(3), 200000, 2)), - ImmutableSet.of(Lifespan.driverGroup(3)), - false))); - // assert that pipeline 2 driver group [3] will have no more drivers - waitUntilEquals(scanOperatorFactory2::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryB::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place - scanOperatorFactory0.getPauser().pause(); - - // add assignment for pipeline 0 driver group 3, and mark driver group 3 as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan0NodeId, - ImmutableSet.of(newScheduledSplit(3, scan0NodeId, Lifespan.driverGroup(3), 300000, 10)), - ImmutableSet.of(Lifespan.driverGroup(3)), - false))); - // assert that pipeline 0 driver group [3] will have no more drivers - waitUntilEquals(scanOperatorFactory0::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryA::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryC::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - // assert that no DriverGroup is fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of()); - // resume operator execution - scanOperatorFactory0.getPauser().resume(); - // assert that partial task result is produced - outputBufferConsumer.consume(10 * 5 * 3, ASSERT_WAIT_TIMEOUT); - // assert that driver group [3] is fully completed - waitUntilEquals(taskContext::getCompletedDriverGroups, ImmutableSet.of(Lifespan.driverGroup(3)), ASSERT_WAIT_TIMEOUT); - - // add assignment for pipeline 2 driver group 7, and mark pipeline as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan2NodeId, - ImmutableSet.of(newScheduledSplit(4, scan2NodeId, Lifespan.driverGroup(7), 400000, 2)), - ImmutableSet.of(Lifespan.driverGroup(7)), - true))); - // assert that pipeline 2 driver group [3, 7] will have no more drivers - waitUntilEquals(scanOperatorFactory2::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryB::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - - // pause operator execution to make sure that - // * operatorFactory will be closed even though operator can't execute - // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place - scanOperatorFactory0.getPauser().pause(); - - // add assignment for pipeline 0 driver group 7, mark pipeline as no more splits - sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( - scan0NodeId, - ImmutableSet.of(newScheduledSplit(5, scan0NodeId, Lifespan.driverGroup(7), 500000, 1000)), - ImmutableSet.of(Lifespan.driverGroup(7)), - true))); - // assert that pipeline 0 driver group [3, 7] will have no more drivers - waitUntilEquals(scanOperatorFactory0::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryA::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryC::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - // assert that pipeline 0 will have no more drivers - waitUntilEquals(scanOperatorFactory0::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryA::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(joinOperatorFactoryC::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that pipeline 1 driver group [3, 7] will have no more drivers - waitUntilEquals(joinOperatorFactoryB::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryA::getDriverGroupsWithNoMoreOperators, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - // assert that pipeline 1 will have no more drivers - // (Unpartitioned grouped pipelines will have no more driver instances when there can be no more driver groups.) - waitUntilEquals(joinOperatorFactoryB::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryA::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that pipeline 2 will have no more drivers - // note: One could argue that this should have happened as soon as pipeline 2 driver group 7 is marked as noMoreSplits. - // This is not how SqlTaskExecution is currently implemented. And such a delay in closing DriverFactory does not matter much. - waitUntilEquals(scanOperatorFactory2::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - waitUntilEquals(buildOperatorFactoryB::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); - // assert that driver group [3] (but not 7) is fully completed - assertEquals(taskContext.getCompletedDriverGroups(), ImmutableSet.of(Lifespan.driverGroup(3))); - // resume operator execution - scanOperatorFactory0.getPauser().resume(); - // assert that partial task result is produced - outputBufferConsumer.consume(1000 * 2 * 3, ASSERT_WAIT_TIMEOUT); - outputBufferConsumer.assertBufferComplete(ASSERT_WAIT_TIMEOUT); - // assert that driver group [3, 7] is fully completed - waitUntilEquals(taskContext::getCompletedDriverGroups, ImmutableSet.of(Lifespan.driverGroup(3), Lifespan.driverGroup(7)), ASSERT_WAIT_TIMEOUT); - - break; - default: - throw new UnsupportedOperationException(); - } + // add assignment for pipeline + sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( + TABLE_SCAN_NODE_ID, + ImmutableSet.of(newScheduledSplit(0, TABLE_SCAN_NODE_ID, 100000, 123)), + false))); + // assert that partial task result is produced + outputBufferConsumer.consume(123, ASSERT_WAIT_TIMEOUT); + + // pause operator execution to make sure that + // * operatorFactory will be closed even though operator can't execute + // * completedDriverGroups will NOT include the newly scheduled driver group while pause is in place + testingScanOperatorFactory.getPauser().pause(); + // add assignment for pipeline, mark as no more splits + sqlTaskExecution.addSplitAssignments(ImmutableList.of(new SplitAssignment( + TABLE_SCAN_NODE_ID, + ImmutableSet.of( + newScheduledSplit(1, TABLE_SCAN_NODE_ID, 200000, 300), + newScheduledSplit(2, TABLE_SCAN_NODE_ID, 300000, 200)), + true))); + // assert that pipeline will have no more drivers + waitUntilEquals(testingScanOperatorFactory::isOverallNoMoreOperators, true, ASSERT_WAIT_TIMEOUT); + // resume operator execution + testingScanOperatorFactory.getPauser().resume(); + // assert that task result is produced + outputBufferConsumer.consume(300 + 200, ASSERT_WAIT_TIMEOUT); + outputBufferConsumer.assertBufferComplete(ASSERT_WAIT_TIMEOUT); assertEquals(taskStateMachine.getStateChange(RUNNING).get(10, SECONDS), FLUSHING); outputBufferConsumer.abort(); // complete the task by calling abort on it @@ -687,9 +285,9 @@ public void abort() } } - private ScheduledSplit newScheduledSplit(int sequenceId, PlanNodeId planNodeId, Lifespan lifespan, int begin, int count) + private ScheduledSplit newScheduledSplit(int sequenceId, PlanNodeId planNodeId, int begin, int count) { - return new ScheduledSplit(sequenceId, planNodeId, new Split(CONNECTOR_ID, new TestingSplit(begin, begin + count), lifespan)); + return new ScheduledSplit(sequenceId, planNodeId, new Split(CONNECTOR_ID, new TestingSplit(begin, begin + count))); } public static class Pauser @@ -735,7 +333,6 @@ public static class TestingScanOperatorFactory private final PlanNodeId sourceId; private final Pauser pauser = new Pauser(); - private final Set driverGroupsWithNoMoreOperators = new HashSet<>(); private boolean overallNoMoreOperators; public TestingScanOperatorFactory( @@ -757,29 +354,16 @@ public PlanNodeId getSourceId() public SourceOperator createOperator(DriverContext driverContext) { checkState(!overallNoMoreOperators, "noMoreOperators() has been called"); - checkState(!driverGroupsWithNoMoreOperators.contains(driverContext.getLifespan()), "noMoreOperators(lifespan) has been called"); OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, sourceId, TestingScanOperator.class.getSimpleName()); return new TestingScanOperator(operatorContext, sourceId); } - @Override - public synchronized void noMoreOperators(Lifespan lifespan) - { - checkArgument(!driverGroupsWithNoMoreOperators.contains(lifespan)); - driverGroupsWithNoMoreOperators.add(lifespan); - } - @Override public void noMoreOperators() { overallNoMoreOperators = true; } - public synchronized Set getDriverGroupsWithNoMoreOperators() - { - return ImmutableSet.copyOf(driverGroupsWithNoMoreOperators); - } - public boolean isOverallNoMoreOperators() { return overallNoMoreOperators; @@ -897,416 +481,6 @@ public Page getOutput() } } - public static class TestingBuildOperatorFactory - implements OperatorFactory - { - private final int operatorId; - private final PlanNodeId planNodeId; - private final Pauser pauser = new Pauser(); - - private final Set driverGroupsWithNoMoreOperators = new HashSet<>(); - private boolean overallNoMoreOperators; - private final BuildStates buildStates; - - public TestingBuildOperatorFactory( - int operatorId, - PlanNodeId planNodeId, - BuildStates buildStates) - { - this.operatorId = operatorId; - this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.buildStates = requireNonNull(buildStates, "buildStates is null"); - } - - @Override - public Operator createOperator(DriverContext driverContext) - { - checkState(!overallNoMoreOperators, "noMoreOperators() has been called"); - checkState(!driverGroupsWithNoMoreOperators.contains(driverContext.getLifespan()), "noMoreOperators(lifespan) has been called"); - OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, TestingBuildOperator.class.getSimpleName()); - buildStates.get(driverContext.getLifespan()).incrementPendingBuildCount(); - return new TestingBuildOperator(operatorContext, driverContext.getLifespan()); - } - - @Override - public synchronized void noMoreOperators(Lifespan lifespan) - { - checkArgument(!driverGroupsWithNoMoreOperators.contains(lifespan)); - buildStates.get(lifespan).setNoNewBuilds(); - driverGroupsWithNoMoreOperators.add(lifespan); - } - - @Override - public void noMoreOperators() - { - overallNoMoreOperators = true; - } - - @Override - public OperatorFactory duplicate() - { - throw new UnsupportedOperationException(); - } - - public synchronized Set getDriverGroupsWithNoMoreOperators() - { - return ImmutableSet.copyOf(driverGroupsWithNoMoreOperators); - } - - public boolean isOverallNoMoreOperators() - { - return overallNoMoreOperators; - } - - public Pauser getPauser() - { - return pauser; - } - - public class TestingBuildOperator - implements Operator - { - private final OperatorContext operatorContext; - private final Lifespan lifespan; - - private final List pages = new ArrayList<>(); - - private boolean finishing; - - public TestingBuildOperator( - OperatorContext operatorContext, - Lifespan lifespan) - { - this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); - this.lifespan = requireNonNull(lifespan, "lifespan is null"); - } - - @Override - public OperatorContext getOperatorContext() - { - return operatorContext; - } - - @Override - public void finish() - { - if (finishing) { - return; - } - finishing = true; - buildStates.get(lifespan).addBuildResult(pages); - } - - @Override - public ListenableFuture isBlocked() - { - if (!finishing) { - return NOT_BLOCKED; - } - return buildStates.get(lifespan).getLookupDoneFuture(); - } - - @Override - public boolean isFinished() - { - return finishing && buildStates.get(lifespan).getLookupDoneFuture().isDone(); - } - - @Override - public boolean needsInput() - { - return !finishing; - } - - @Override - public void addInput(Page page) - { - pages.add(page); - } - - @Override - public Page getOutput() - { - return null; - } - } - } - - public static class TestingCrossJoinOperatorFactory - implements OperatorFactory - { - private final int operatorId; - private final PlanNodeId planNodeId; - private final Pauser pauser = new Pauser(); - - private final Set driverGroupsWithNoMoreOperators = new HashSet<>(); - private boolean overallNoMoreOperators; - private final BuildStates buildStates; - - public TestingCrossJoinOperatorFactory( - int operatorId, - PlanNodeId planNodeId, - BuildStates buildStates) - { - this.operatorId = operatorId; - this.planNodeId = requireNonNull(planNodeId, "planNodeId is null"); - this.buildStates = requireNonNull(buildStates, "buildStates is null"); - } - - @Override - public Operator createOperator(DriverContext driverContext) - { - checkState(!overallNoMoreOperators, "noMoreOperators() has been called"); - checkState(!driverGroupsWithNoMoreOperators.contains(driverContext.getLifespan()), "noMoreOperators(lifespan) has been called"); - OperatorContext operatorContext = driverContext.addOperatorContext(operatorId, planNodeId, TestingCrossJoinOperator.class.getSimpleName()); - buildStates.get(driverContext.getLifespan()).incrementPendingLookupCount(); - return new TestingCrossJoinOperator(operatorContext, driverContext.getLifespan()); - } - - @Override - public synchronized void noMoreOperators(Lifespan lifespan) - { - checkArgument(!driverGroupsWithNoMoreOperators.contains(lifespan)); - buildStates.setNoNewLookups(lifespan); - driverGroupsWithNoMoreOperators.add(lifespan); - } - - @Override - public void noMoreOperators() - { - buildStates.setNoNewLookups(); - overallNoMoreOperators = true; - } - - @Override - public OperatorFactory duplicate() - { - throw new UnsupportedOperationException(); - } - - public synchronized Set getDriverGroupsWithNoMoreOperators() - { - return ImmutableSet.copyOf(driverGroupsWithNoMoreOperators); - } - - public boolean isOverallNoMoreOperators() - { - return overallNoMoreOperators; - } - - public Pauser getPauser() - { - return pauser; - } - - public class TestingCrossJoinOperator - implements Operator - { - private final OperatorContext operatorContext; - private final Lifespan lifespan; - - private final ListenableFuture multiplierFuture; - private final ListenableFuture blockedFutureView; - private final Queue pages = new ArrayDeque<>(); - private boolean finishing; - - public TestingCrossJoinOperator( - OperatorContext operatorContext, - Lifespan lifespan) - { - this.operatorContext = requireNonNull(operatorContext, "operatorContext is null"); - this.lifespan = requireNonNull(lifespan, "lifespan is null"); - multiplierFuture = Futures.transform(buildStates.get(lifespan).getPagesFuture(), buildPages -> { - requireNonNull(buildPages, "buildPages is null"); - return buildPages.stream() - .mapToInt(Page::getPositionCount) - .sum(); - }, directExecutor()); - blockedFutureView = asVoid(multiplierFuture); - } - - @Override - public OperatorContext getOperatorContext() - { - return operatorContext; - } - - @Override - public void finish() - { - if (finishing) { - return; - } - finishing = true; - } - - @Override - public ListenableFuture isBlocked() - { - return blockedFutureView; - } - - @Override - public boolean isFinished() - { - return finishing && pages.isEmpty(); - } - - @Override - public boolean needsInput() - { - return !finishing && multiplierFuture.isDone(); - } - - @Override - public void addInput(Page page) - { - int multiplier = getFutureValue(multiplierFuture); - for (int i = 0; i < multiplier; i++) { - pages.add(page); - } - } - - @Override - public Page getOutput() - { - Page result = pages.poll(); - if (isFinished() && pages.isEmpty()) { - buildStates.get(lifespan).decrementPendingLookupCount(); - } - return result; - } - } - } - - private static ListenableFuture asVoid(ListenableFuture future) - { - return Futures.transform(future, v -> null, directExecutor()); - } - - private static class BuildStates - { - private final HashMap buildStatesMap = new HashMap<>(); - private final boolean grouped; - - public BuildStates(PipelineExecutionStrategy executionStrategy) - { - this.grouped = executionStrategy == GROUPED_EXECUTION; - } - - public synchronized BuildState get(Lifespan lifespan) - { - if (grouped) { - return buildStatesMap.computeIfAbsent(lifespan, ignored -> new BuildState()); - } - else { - return buildStatesMap.computeIfAbsent(Lifespan.taskWide(), ignored -> new BuildState()); - } - } - - public void setNoNewLookups(Lifespan lifespan) - { - // If it's grouped execution, knowing no new probe operators will be created for a group - // should be immediately propagated to build state, so that corresponding build - // can move forward (e.g. clean up). - // On the other hand, if there's a single build, knowing no new probe operators will be - // created for a group has no immediate use. - if (grouped) { - get(lifespan).setNoNewLookups(); - } - } - - public void setNoNewLookups() - { - if (!grouped) { - get(Lifespan.taskWide()).setNoNewLookups(); - } - } - - private static class BuildState - { - private final SettableFuture> pagesFuture = SettableFuture.create(); - private final SettableFuture lookupDoneFuture = SettableFuture.create(); - - private final List pages = new ArrayList<>(); - private int pendingBuildCount; - private boolean noNewBuilds; - private int pendingLookupCount; - private boolean noNewLookups; - - public synchronized void addBuildResult(List newPages) - { - checkState(!pagesFuture.isDone()); - pages.addAll(newPages); - pendingBuildCount--; - checkAllBuildsDone(); - } - - public synchronized void incrementPendingBuildCount() - { - checkState(!noNewBuilds); - pendingBuildCount++; - } - - public synchronized void setNoNewBuilds() - { - if (noNewBuilds) { - return; - } - checkState(!pagesFuture.isDone()); - noNewBuilds = true; - checkAllBuildsDone(); - } - - public synchronized void checkAllBuildsDone() - { - if (pendingBuildCount == 0 && noNewBuilds) { - pagesFuture.set(pages); - } - } - - public ListenableFuture> getPagesFuture() - { - return pagesFuture; - } - - public synchronized void decrementPendingLookupCount() - { - checkState(!lookupDoneFuture.isDone()); - pendingLookupCount--; - checkAllLookupsDone(); - } - - public synchronized void incrementPendingLookupCount() - { - checkState(!noNewLookups); - pendingLookupCount++; - } - - synchronized void setNoNewLookups() - { - if (noNewLookups) { - return; - } - checkState(!lookupDoneFuture.isDone()); - noNewLookups = true; - checkAllLookupsDone(); - } - - public synchronized void checkAllLookupsDone() - { - if (pendingLookupCount == 0 && noNewLookups) { - lookupDoneFuture.set(null); - } - } - - public ListenableFuture getLookupDoneFuture() - { - return lookupDoneFuture; - } - } - } - public static class TestingSplit implements ConnectorSplit { diff --git a/core/trino-main/src/test/java/io/trino/execution/TestStageStateMachine.java b/core/trino-main/src/test/java/io/trino/execution/TestStageStateMachine.java index 9693c6d7b064..3d52a5241eb8 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestStageStateMachine.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestStageStateMachine.java @@ -35,7 +35,6 @@ import java.util.concurrent.ExecutorService; import static io.airlift.concurrent.Threads.daemonThreadsNamed; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; @@ -253,7 +252,6 @@ private static PlanFragment createValuesPlan() SOURCE_DISTRIBUTION, ImmutableList.of(valuesNodeId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol)), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); diff --git a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java index c85099e73fdf..db478391af51 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestingRemoteTaskFactory.java @@ -168,7 +168,6 @@ public TaskStatus getTaskStatus() state, location, nodeId, - ImmutableSet.of(), failures, 0, 0, @@ -217,11 +216,6 @@ public Set getNoMoreSplits() return ImmutableSet.copyOf(noMoreSplits); } - @Override - public void noMoreSplits(PlanNodeId sourceId, Lifespan lifespan) - { - } - @Override public synchronized void setOutputBuffers(OutputBuffers outputBuffers) { diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFaultTolerantStageScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFaultTolerantStageScheduler.java index bf495e4c0a3d..31545ec7d7a6 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFaultTolerantStageScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestFaultTolerantStageScheduler.java @@ -27,7 +27,6 @@ import io.trino.client.NodeVersion; import io.trino.connector.CatalogName; import io.trino.cost.StatsAndCosts; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap; import io.trino.execution.RemoteTaskFactory; import io.trino.execution.SqlStage; @@ -80,7 +79,6 @@ import static io.airlift.units.DataSize.Unit.GIGABYTE; import static io.airlift.units.DataSize.Unit.MEGABYTE; import static io.trino.operator.RetryPolicy.TASK; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; @@ -311,12 +309,12 @@ public void testTasksWaitingForNodes() { TestingRemoteTaskFactory remoteTaskFactory = new TestingRemoteTaskFactory(); List splits = ImmutableList.of( - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort())), Lifespan.taskWide()), // 0 - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort())), Lifespan.taskWide()), // 1 - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort())), Lifespan.taskWide()), // 2 - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_2.getHostAndPort())), Lifespan.taskWide()), // 3 - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort())), Lifespan.taskWide()), // 4 - new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_3.getHostAndPort())), Lifespan.taskWide())); // 5 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort()))), // 0 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort()))), // 1 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort()))), // 2 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_2.getHostAndPort()))), // 3 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_1.getHostAndPort()))), // 4 + new Split(CATALOG, new TestingSplit(false, ImmutableList.of(NODE_3.getHostAndPort())))); // 5 TestingTaskSourceFactory taskSourceFactory = new TestingTaskSourceFactory(Optional.of(CATALOG), splits, 2); TestingNodeSupplier nodeSupplier = TestingNodeSupplier.create(ImmutableMap.of( NODE_1, ImmutableList.of(CATALOG), @@ -1043,7 +1041,6 @@ private PlanFragment createPlanFragment() SOURCE_DISTRIBUTION, ImmutableList.of(TABLE_SCAN_NODE_ID), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(probeColumnSymbol, buildColumnSymbol)), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); } @@ -1055,7 +1052,7 @@ private static TestingTaskSourceFactory createTaskSourceFactory(int splitCount, private static List createSplits(int count) { - return ImmutableList.copyOf(limit(cycle(new Split(CATALOG, createRemoteSplit(), Lifespan.taskWide())), count)); + return ImmutableList.copyOf(limit(cycle(new Split(CATALOG, createRemoteSplit())), count)); } private static TaskId getTaskId(int partitionId, int attemptId) diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java index 25447b0c2311..16947e2619be 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestSourcePartitionedScheduler.java @@ -42,7 +42,6 @@ import io.trino.operator.RetryPolicy; import io.trino.server.DynamicFilterService; import io.trino.spi.QueryId; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.DynamicFilter; @@ -81,7 +80,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.function.Supplier; -import static com.google.common.base.Preconditions.checkArgument; import static io.airlift.concurrent.Threads.daemonThreadsNamed; import static io.trino.SessionTestUtils.TEST_SESSION; import static io.trino.execution.scheduler.NodeSchedulerConfig.SplitsBalancingPolicy.NODE; @@ -93,9 +91,7 @@ import static io.trino.execution.scheduler.StageExecution.State.SCHEDULING; import static io.trino.metadata.FunctionManager.createTestingFunctionManager; import static io.trino.metadata.MetadataManager.createTestMetadataManager; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.StandardErrorCode.NO_NODES_AVAILABLE; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.DynamicFilters.createDynamicFilterExpression; @@ -693,7 +689,6 @@ private static PlanFragment createFragment() SOURCE_DISTRIBUTION, ImmutableList.of(TABLE_SCAN_NODE_ID), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol)), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); } @@ -703,7 +698,7 @@ private static ConnectorSplitSource createBlockedSplitSource() return new ConnectorSplitSource() { @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { return new CompletableFuture<>(); } @@ -783,9 +778,8 @@ synchronized void addSplits(int count) } @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { - checkArgument(partitionHandle.equals(NOT_PARTITIONED), "partitionHandle must be NOT_PARTITIONED"); return notEmptyFuture .thenApply(x -> getBatch(maxSize)) .thenApply(splits -> new ConnectorSplitBatch(splits, isFinished())); diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestStageTaskSourceFactory.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestStageTaskSourceFactory.java index 59134ca1d828..5819a1e60a66 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestStageTaskSourceFactory.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestStageTaskSourceFactory.java @@ -26,7 +26,6 @@ import com.google.common.util.concurrent.SettableFuture; import io.airlift.units.DataSize; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.execution.TableExecuteContextManager; import io.trino.execution.scheduler.StageTaskSourceFactory.ArbitraryDistributionTaskSource; import io.trino.execution.scheduler.StageTaskSourceFactory.HashDistributionTaskSource; @@ -928,17 +927,17 @@ private static SourceDistributionTaskSource createSourceDistributionTaskSource( private static Split createSplit(int id, String... addresses) { - return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.empty(), addressesList(addresses)), Lifespan.taskWide()); + return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.empty(), addressesList(addresses))); } private static Split createWeightedSplit(int id, long weight, String... addresses) { - return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.empty(), addressesList(addresses), weight), Lifespan.taskWide()); + return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.empty(), addressesList(addresses), weight)); } private static Split createBucketedSplit(int id, int bucket) { - return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.of(bucket), Optional.empty()), Lifespan.taskWide()); + return new Split(CATALOG, new TestingConnectorSplit(id, OptionalInt.of(bucket), Optional.empty())); } private List readAllTasks(TaskSource taskSource) diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestingSplitSource.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestingSplitSource.java index 8fb3afae1be2..2a84f03c92ec 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/TestingSplitSource.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/TestingSplitSource.java @@ -17,9 +17,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.split.SplitSource; import java.util.Iterator; @@ -66,7 +64,7 @@ public CatalogName getCatalogName() } @Override - public ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize) + public ListenableFuture getNextBatch(int maxSize) { if (isFinished()) { return immediateFuture(new SplitBatch(ImmutableList.of(), true)); diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java index ce33f622fd3e..5b9069567a98 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/PlanUtils.java @@ -37,7 +37,6 @@ import java.util.stream.Stream; import static com.google.common.collect.ImmutableList.toImmutableList; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SOURCE_DISTRIBUTION; @@ -230,7 +229,6 @@ private static PlanFragment createFragment(PlanNode planNode) SOURCE_DISTRIBUTION, ImmutableList.of(planNode.getId()), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), planNode.getOutputSymbols()), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); } diff --git a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java index 7aad6ba31ffe..e2b25fda30c4 100644 --- a/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java +++ b/core/trino-main/src/test/java/io/trino/execution/scheduler/policy/TestPhasedExecutionSchedule.java @@ -17,7 +17,6 @@ import com.google.common.collect.Multimap; import com.google.common.util.concurrent.ListenableFuture; import io.trino.execution.ExecutionFailureInfo; -import io.trino.execution.Lifespan; import io.trino.execution.RemoteTask; import io.trino.execution.StageId; import io.trino.execution.StateMachine.StateChangeListener; @@ -40,7 +39,6 @@ import java.util.List; import java.util.Optional; import java.util.Set; -import java.util.function.Consumer; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; @@ -305,12 +303,6 @@ public void transitionToSchedulingSplits() throw new UnsupportedOperationException(); } - @Override - public void addCompletedDriverGroupsChangedListener(Consumer> newlyCompletedDriverGroupConsumer) - { - throw new UnsupportedOperationException(); - } - @Override public TaskLifecycleListener getTaskLifecycleListener() { @@ -348,7 +340,7 @@ public void recordGetSplitTime(long start) } @Override - public Optional scheduleTask(InternalNode node, int partition, Multimap initialSplits, Multimap noMoreSplitsForLifespan) + public Optional scheduleTask(InternalNode node, int partition, Multimap initialSplits) { throw new UnsupportedOperationException(); } diff --git a/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java b/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java index fc70e13c1825..b35a14e1fc99 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestLeastWastedEffortTaskLowMemoryKiller.java @@ -224,7 +224,6 @@ private static TaskInfo buildTaskInfo(TaskId taskId, TaskState state, Duration s state, URI.create("fake://task/" + taskId + "/node/some_node"), "some_node", - ImmutableSet.of(), ImmutableList.of(), 0, 0, diff --git a/core/trino-main/src/test/java/io/trino/memory/TestMemoryBlocking.java b/core/trino-main/src/test/java/io/trino/memory/TestMemoryBlocking.java index 1ea01b2dbbcd..5053c80f34e2 100644 --- a/core/trino-main/src/test/java/io/trino/memory/TestMemoryBlocking.java +++ b/core/trino-main/src/test/java/io/trino/memory/TestMemoryBlocking.java @@ -19,7 +19,6 @@ import io.airlift.units.DataSize; import io.airlift.units.Duration; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.execution.ScheduledSplit; import io.trino.execution.SplitAssignment; import io.trino.execution.StageId; @@ -115,7 +114,7 @@ public void testTableScanMemoryBlocking() Driver driver = Driver.createDriver(driverContext, source, sink); assertSame(driver.getDriverContext(), driverContext); assertFalse(driver.isFinished()); - Split testSplit = new Split(new CatalogName("test"), new TestSplit(), Lifespan.taskWide()); + Split testSplit = new Split(new CatalogName("test"), new TestSplit()); driver.updateSplitAssignment(new SplitAssignment(sourceId, ImmutableSet.of(new ScheduledSplit(0, sourceId, testSplit)), true)); ListenableFuture blocked = driver.processForDuration(new Duration(1, NANOSECONDS)); diff --git a/core/trino-main/src/test/java/io/trino/operator/BenchmarkScanFilterAndProjectOperator.java b/core/trino-main/src/test/java/io/trino/operator/BenchmarkScanFilterAndProjectOperator.java index b386d973c3da..aae60d6afea5 100644 --- a/core/trino-main/src/test/java/io/trino/operator/BenchmarkScanFilterAndProjectOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/BenchmarkScanFilterAndProjectOperator.java @@ -19,7 +19,6 @@ import io.trino.SequencePageBuilder; import io.trino.Session; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; import io.trino.operator.ScanFilterAndProjectOperator.ScanFilterAndProjectOperatorFactory; import io.trino.operator.project.CursorProcessor; @@ -253,7 +252,7 @@ public List benchmarkColumnOriented(Context context) SourceOperator operator = (SourceOperator) context.getOperatorFactory().createOperator(driverContext); ImmutableList.Builder outputPages = ImmutableList.builder(); - operator.addSplit(new Split(new CatalogName("test"), createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), createLocalSplit())); operator.noMoreSplits(); for (int loops = 0; !operator.isFinished() && loops < 1_000_000; loops++) { diff --git a/core/trino-main/src/test/java/io/trino/operator/TestDriver.java b/core/trino-main/src/test/java/io/trino/operator/TestDriver.java index b4ea218fad8a..022038917d66 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestDriver.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestDriver.java @@ -20,7 +20,6 @@ import com.google.common.util.concurrent.Uninterruptibles; import io.airlift.units.Duration; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.execution.ScheduledSplit; import io.trino.execution.SplitAssignment; import io.trino.memory.context.LocalMemoryContext; @@ -302,7 +301,7 @@ public void testBrokenOperatorAddSource() private static Split newMockSplit() { - return new Split(new CatalogName("test"), new MockSplit(), Lifespan.taskWide()); + return new Split(new CatalogName("test"), new MockSplit()); } private PageConsumerOperator createSinkOperator(List types) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestDriverStats.java b/core/trino-main/src/test/java/io/trino/operator/TestDriverStats.java index 3e1827a14023..c9db915016cd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestDriverStats.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestDriverStats.java @@ -18,7 +18,6 @@ import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; import io.airlift.units.Duration; -import io.trino.execution.Lifespan; import org.joda.time.DateTime; import org.testng.annotations.Test; @@ -30,8 +29,6 @@ public class TestDriverStats { public static final DriverStats EXPECTED = new DriverStats( - Lifespan.driverGroup(21), - new DateTime(1), new DateTime(2), new DateTime(3), @@ -86,8 +83,6 @@ public void testJson() public static void assertExpectedDriverStats(DriverStats actual) { - assertEquals(actual.getLifespan(), Lifespan.driverGroup(21)); - assertEquals(actual.getCreateTime(), new DateTime(1, UTC)); assertEquals(actual.getStartTime(), new DateTime(2, UTC)); assertEquals(actual.getEndTime(), new DateTime(3, UTC)); diff --git a/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java index a4eef73d3e88..1a15ce8fa149 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestExchangeOperator.java @@ -23,7 +23,6 @@ import io.airlift.units.Duration; import io.trino.FeaturesConfig.DataIntegrityVerification; import io.trino.exchange.ExchangeManagerRegistry; -import io.trino.execution.Lifespan; import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.execution.buffer.PagesSerdeFactory; @@ -154,7 +153,7 @@ public void testSimple() private static Split newRemoteSplit(TaskId taskId) { - return new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(taskId, "http://localhost/" + taskId)), Lifespan.taskWide()); + return new Split(REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(taskId, "http://localhost/" + taskId))); } @Test diff --git a/core/trino-main/src/test/java/io/trino/operator/TestMergeOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestMergeOperator.java index a9dbf7d1ada2..66c18fe28a44 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestMergeOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestMergeOperator.java @@ -22,7 +22,6 @@ import io.airlift.node.NodeInfo; import io.trino.FeaturesConfig; import io.trino.exchange.ExchangeManagerRegistry; -import io.trino.execution.Lifespan; import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.execution.buffer.PagesSerdeFactory; @@ -364,7 +363,7 @@ private MergeOperator createMergeOperator(List sourceTypes, List private static Split createRemoteSplit(TaskId taskId) { - return new Split(ExchangeOperator.REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(taskId, "http://localhost/" + taskId)), Lifespan.taskWide()); + return new Split(ExchangeOperator.REMOTE_CONNECTOR_ID, new RemoteSplit(new DirectExchangeInput(taskId, "http://localhost/" + taskId))); } private static List pullAvailablePages(Operator operator) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestScanFilterAndProjectOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestScanFilterAndProjectOperator.java index 5b54ab7db26e..f331e435e6ca 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestScanFilterAndProjectOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestScanFilterAndProjectOperator.java @@ -18,7 +18,6 @@ import io.trino.SequencePageBuilder; import io.trino.block.BlockAssertions; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.FunctionManager; import io.trino.metadata.InternalFunctionBundle; import io.trino.metadata.Split; @@ -117,7 +116,7 @@ public void testPageSource() 0); SourceOperator operator = factory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); @@ -160,7 +159,7 @@ public void testPageSourceMergeOutput() 2); SourceOperator operator = factory.createOperator(newDriverContext()); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); List actual = toPages(operator); @@ -205,7 +204,7 @@ public void testPageSourceLazyLoad() 0); SourceOperator operator = factory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(BIGINT), ImmutableList.of(new Page(inputBlock))); @@ -240,7 +239,7 @@ public void testRecordCursorSource() 0); SourceOperator operator = factory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); MaterializedResult expected = toMaterializedResult(driverContext.getSession(), ImmutableList.of(VARCHAR), ImmutableList.of(input)); @@ -292,7 +291,7 @@ public void testPageYield() 0); SourceOperator operator = factory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); // In the below loop we yield for every cell: 20 X 1000 times @@ -357,7 +356,7 @@ public void testRecordCursorYield() 0); SourceOperator operator = factory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); // start driver; get null value due to yield for the first 15 times diff --git a/core/trino-main/src/test/java/io/trino/operator/TestWorkProcessorPipelineSourceOperator.java b/core/trino-main/src/test/java/io/trino/operator/TestWorkProcessorPipelineSourceOperator.java index 089b37faf1d5..bf130bf54983 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestWorkProcessorPipelineSourceOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestWorkProcessorPipelineSourceOperator.java @@ -46,7 +46,6 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.RowPagesBuilder.rowPagesBuilder; import static io.trino.SessionTestUtils.TEST_SESSION; -import static io.trino.execution.Lifespan.taskWide; import static io.trino.operator.WorkProcessorAssertion.transformationFrom; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.testing.TestingSplit.createLocalSplit; @@ -299,8 +298,7 @@ private Split createSplit() { return new Split( new CatalogName("catalog_name"), - createLocalSplit(), - taskWide()); + createLocalSplit()); } private Page createPage(int pageNumber) diff --git a/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java b/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java index 73a89d0ea2fe..db2930852237 100644 --- a/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java +++ b/core/trino-main/src/test/java/io/trino/operator/TestingOperatorContext.java @@ -14,7 +14,6 @@ package io.trino.operator; import com.google.common.util.concurrent.MoreExecutors; -import io.trino.execution.Lifespan; import io.trino.memory.context.MemoryTrackingContext; import io.trino.sql.planner.plan.PlanNodeId; import io.trino.testing.TestingSession; @@ -53,7 +52,6 @@ public static OperatorContext create(ScheduledExecutorService scheduledExecutor) executor, scheduledExecutor, pipelineMemoryContext, - Lifespan.taskWide(), 0L); OperatorContext operatorContext = driverContext.addOperatorContext( diff --git a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java index bd90b26bd922..f2da203531fd 100644 --- a/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java +++ b/core/trino-main/src/test/java/io/trino/operator/exchange/TestLocalExchange.java @@ -19,7 +19,6 @@ import io.trino.SequencePageBuilder; import io.trino.Session; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap; import io.trino.execution.scheduler.NodeScheduler; import io.trino.execution.scheduler.NodeSchedulerConfig; @@ -27,10 +26,7 @@ import io.trino.metadata.InMemoryNodeManager; import io.trino.operator.InterpretedHashGenerator; import io.trino.operator.PageAssertions; -import io.trino.operator.PipelineExecutionStrategy; -import io.trino.operator.exchange.LocalExchange.LocalExchangeFactory; import io.trino.operator.exchange.LocalExchange.LocalExchangeSinkFactory; -import io.trino.operator.exchange.LocalExchange.LocalExchangeSinkFactoryId; import io.trino.spi.Page; import io.trino.spi.connector.BucketFunction; import io.trino.spi.connector.ConnectorBucketNodeMap; @@ -47,7 +43,6 @@ import io.trino.type.BlockTypeOperators; import io.trino.util.FinalizerService; import org.testng.annotations.BeforeMethod; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.util.List; @@ -58,8 +53,6 @@ import java.util.function.ToIntFunction; import static com.google.common.base.Preconditions.checkArgument; -import static io.trino.operator.PipelineExecutionStrategy.GROUPED_EXECUTION; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.connector.ConnectorBucketNodeMap.createBucketNodeMap; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -69,7 +62,6 @@ import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_PASSTHROUGH_DISTRIBUTION; import static io.trino.sql.planner.SystemPartitioningHandle.SINGLE_DISTRIBUTION; import static io.trino.testing.TestingSession.testSessionBuilder; -import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; @@ -105,41 +97,32 @@ public void setUp() }); } - @DataProvider - public static Object[][] executionStrategy() - { - return new Object[][] {{UNGROUPED_EXECUTION}, {GROUPED_EXECUTION}}; - } - - @Test(dataProvider = "executionStrategy") - public void testGatherSingleWriter(PipelineExecutionStrategy executionStrategy) + @Test + public void testGatherSingleWriter() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - SINGLE_DISTRIBUTION, 8, - TYPES, + SINGLE_DISTRIBUTION, ImmutableList.of(), + TYPES, Optional.empty(), - executionStrategy, DataSize.ofBytes(retainedSizeOfPages(99)), TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 1); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSource source = exchange.getSource(0); assertSource(source, 0); LocalExchangeSink sink = sinkFactory.createSink(); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); assertSinkCanWrite(sink); assertSource(source, 0); @@ -186,34 +169,31 @@ public void testGatherSingleWriter(PipelineExecutionStrategy executionStrategy) }); } - @Test(dataProvider = "executionStrategy") - public void testBroadcast(PipelineExecutionStrategy executionStrategy) + @Test + public void testBroadcast() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_BROADCAST_DISTRIBUTION, 2, - TYPES, + FIXED_BROADCAST_DISTRIBUTION, ImmutableList.of(), + TYPES, Optional.empty(), - executionStrategy, LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sinkA = sinkFactory.createSink(); assertSinkCanWrite(sinkA); LocalExchangeSink sinkB = sinkFactory.createSink(); assertSinkCanWrite(sinkB); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -276,32 +256,29 @@ public void testBroadcast(PipelineExecutionStrategy executionStrategy) }); } - @Test(dataProvider = "executionStrategy") - public void testRandom(PipelineExecutionStrategy executionStrategy) + @Test + public void testRandom() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_ARBITRARY_DISTRIBUTION, 2, - TYPES, + FIXED_ARBITRARY_DISTRIBUTION, ImmutableList.of(), + TYPES, Optional.empty(), - executionStrategy, LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sink = sinkFactory.createSink(); assertSinkCanWrite(sink); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -327,35 +304,31 @@ public void testRandom(PipelineExecutionStrategy executionStrategy) }); } - @Test(dataProvider = "executionStrategy") - public void testPassthrough(PipelineExecutionStrategy executionStrategy) + @Test + public void testPassthrough() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_PASSTHROUGH_DISTRIBUTION, 2, - TYPES, + FIXED_PASSTHROUGH_DISTRIBUTION, ImmutableList.of(), + TYPES, Optional.empty(), - executionStrategy, DataSize.ofBytes(retainedSizeOfPages(1)), TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sinkA = sinkFactory.createSink(); LocalExchangeSink sinkB = sinkFactory.createSink(); assertSinkCanWrite(sinkA); assertSinkCanWrite(sinkB); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -397,32 +370,29 @@ public void testPassthrough(PipelineExecutionStrategy executionStrategy) }); } - @Test(dataProvider = "executionStrategy") - public void testPartition(PipelineExecutionStrategy executionStrategy) + @Test + public void testPartition() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_HASH_DISTRIBUTION, 2, - TYPES, + FIXED_HASH_DISTRIBUTION, ImmutableList.of(0), + TYPES, Optional.empty(), - executionStrategy, LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sink = sinkFactory.createSink(); assertSinkCanWrite(sink); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -466,8 +436,8 @@ public void testPartition(PipelineExecutionStrategy executionStrategy) }); } - @Test(dataProvider = "executionStrategy") - public void testPartitionCustomPartitioning(PipelineExecutionStrategy executionStrategy) + @Test + public void testPartitionCustomPartitioning() { ConnectorPartitioningHandle connectorPartitioningHandle = new ConnectorPartitioningHandle() {}; ConnectorNodePartitioningProvider connectorNodePartitioningProvider = new ConnectorNodePartitioningProvider() @@ -504,29 +474,26 @@ public BucketFunction getBucketFunction(ConnectorTransactionHandle transactionHa Optional.of(new CatalogName("foo")), Optional.of(TestingTransactionHandle.create()), connectorPartitioningHandle); - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - partitioningHandle, 2, - types, + partitioningHandle, ImmutableList.of(1), + types, Optional.empty(), - executionStrategy, LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sink = sinkFactory.createSink(); assertSinkCanWrite(sink); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(1); assertSource(sourceA, 0); @@ -554,36 +521,33 @@ public BucketFunction getBucketFunction(ConnectorTransactionHandle transactionHa }); } - @Test(dataProvider = "executionStrategy") - public void writeUnblockWhenAllReadersFinish(PipelineExecutionStrategy executionStrategy) + @Test + public void writeUnblockWhenAllReadersFinish() { ImmutableList types = ImmutableList.of(BIGINT); - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_BROADCAST_DISTRIBUTION, 2, - types, + FIXED_BROADCAST_DISTRIBUTION, ImmutableList.of(), + types, Optional.empty(), - executionStrategy, LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sinkA = sinkFactory.createSink(); assertSinkCanWrite(sinkA); LocalExchangeSink sinkB = sinkFactory.createSink(); assertSinkCanWrite(sinkB); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -605,34 +569,31 @@ public void writeUnblockWhenAllReadersFinish(PipelineExecutionStrategy execution }); } - @Test(dataProvider = "executionStrategy") - public void writeUnblockWhenAllReadersFinishAndPagesConsumed(PipelineExecutionStrategy executionStrategy) + @Test + public void writeUnblockWhenAllReadersFinishAndPagesConsumed() { - LocalExchangeFactory localExchangeFactory = new LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, SESSION, - FIXED_BROADCAST_DISTRIBUTION, 2, - TYPES, + FIXED_BROADCAST_DISTRIBUTION, ImmutableList.of(), + TYPES, Optional.empty(), - executionStrategy, DataSize.ofBytes(1), TYPE_OPERATOR_FACTORY); - LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); - run(localExchangeFactory, executionStrategy, exchange -> { + run(localExchange, exchange -> { assertEquals(exchange.getBufferCount(), 2); assertExchangeTotalBufferedBytes(exchange, 0); - LocalExchangeSinkFactory sinkFactory = exchange.getSinkFactory(localExchangeSinkFactoryId); + LocalExchangeSinkFactory sinkFactory = exchange.createSinkFactory(); + sinkFactory.noMoreSinkFactories(); LocalExchangeSink sinkA = sinkFactory.createSink(); assertSinkCanWrite(sinkA); LocalExchangeSink sinkB = sinkFactory.createSink(); assertSinkCanWrite(sinkB); sinkFactory.close(); - sinkFactory.noMoreSinkFactories(); LocalExchangeSource sourceA = exchange.getSource(0); assertSource(sourceA, 0); @@ -672,57 +633,9 @@ public void writeUnblockWhenAllReadersFinishAndPagesConsumed(PipelineExecutionSt }); } - @Test - public void testMismatchedExecutionStrategy() - { - // If sink/source didn't create a matching set of exchanges, operators will block forever, - // waiting for the other half that will never show up. - // The most common reason of mismatch is when one of sink/source created the wrong kind of local exchange. - // In such case, we want to fail loudly. - LocalExchangeFactory ungroupedLocalExchangeFactory = new LocalExchangeFactory( - nodePartitioningManager, - SESSION, - FIXED_HASH_DISTRIBUTION, - 2, - TYPES, - ImmutableList.of(0), - Optional.empty(), - UNGROUPED_EXECUTION, - LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, - TYPE_OPERATOR_FACTORY); - assertThatThrownBy(() -> ungroupedLocalExchangeFactory.getLocalExchange(Lifespan.driverGroup(3))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("LocalExchangeFactory is declared as UNGROUPED_EXECUTION. Driver-group exchange cannot be created."); - - LocalExchangeFactory groupedLocalExchangeFactory = new LocalExchangeFactory( - nodePartitioningManager, - SESSION, - FIXED_HASH_DISTRIBUTION, - 2, - TYPES, - ImmutableList.of(0), - Optional.empty(), - GROUPED_EXECUTION, - LOCAL_EXCHANGE_MAX_BUFFERED_BYTES, - TYPE_OPERATOR_FACTORY); - assertThatThrownBy(() -> groupedLocalExchangeFactory.getLocalExchange(Lifespan.taskWide())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("LocalExchangeFactory is declared as GROUPED_EXECUTION. Task-wide exchange cannot be created."); - } - - private void run(LocalExchangeFactory localExchangeFactory, PipelineExecutionStrategy pipelineExecutionStrategy, Consumer test) + private void run(LocalExchange localExchange, Consumer test) { - switch (pipelineExecutionStrategy) { - case UNGROUPED_EXECUTION: - test.accept(localExchangeFactory.getLocalExchange(Lifespan.taskWide())); - return; - case GROUPED_EXECUTION: - test.accept(localExchangeFactory.getLocalExchange(Lifespan.driverGroup(1))); - test.accept(localExchangeFactory.getLocalExchange(Lifespan.driverGroup(12))); - test.accept(localExchangeFactory.getLocalExchange(Lifespan.driverGroup(23))); - return; - } - throw new IllegalArgumentException("Unknown pipelineExecutionStrategy"); + test.accept(localExchange); } private static void assertSource(LocalExchangeSource source, int pageCount) diff --git a/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java b/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java index 621ea2e77905..3c588e5611fe 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/BenchmarkHashBuildAndJoinOperators.java @@ -19,7 +19,6 @@ import io.airlift.units.DataSize; import io.trino.RowPagesBuilder; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; import io.trino.operator.InterpretedHashGenerator; import io.trino.operator.Operator; @@ -389,7 +388,7 @@ private static void buildHash(BuildContext buildContext, JoinBridgeManager lookupSourceProvider = lookupSourceFactory.createLookupSourceProvider(); for (Operator operator : operators) { operator.finish(); diff --git a/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java b/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java index ecb5d812798e..6880661b0a0c 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/JoinTestUtils.java @@ -18,7 +18,6 @@ import com.google.common.util.concurrent.ListenableFuture; import io.airlift.units.DataSize; import io.trino.RowPagesBuilder; -import io.trino.execution.Lifespan; import io.trino.memory.context.LocalMemoryContext; import io.trino.operator.Driver; import io.trino.operator.DriverContext; @@ -64,7 +63,6 @@ import static com.google.common.util.concurrent.Futures.immediateVoidFuture; import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.sql.planner.SystemPartitioningHandle.FIXED_HASH_DISTRIBUTION; import static java.util.Objects.requireNonNull; @@ -143,36 +141,34 @@ public static BuildSideSetup setupBuildSide( int partitionCount = parallelBuild ? PARTITION_COUNT : 1; List hashChannels = buildPages.getHashChannels().orElseThrow(); - LocalExchange.LocalExchangeFactory localExchangeFactory = new LocalExchange.LocalExchangeFactory( + LocalExchange localExchange = new LocalExchange( nodePartitioningManager, taskContext.getSession(), - FIXED_HASH_DISTRIBUTION, partitionCount, - buildPages.getTypes(), + FIXED_HASH_DISTRIBUTION, hashChannels, + buildPages.getTypes(), buildPages.getHashChannel(), - UNGROUPED_EXECUTION, DataSize.of(32, DataSize.Unit.MEGABYTE), TYPE_OPERATOR_FACTORY); - LocalExchange.LocalExchangeSinkFactoryId localExchangeSinkFactoryId = localExchangeFactory.newSinkFactoryId(); - localExchangeFactory.noMoreSinkFactories(); // collect input data into the partitioned exchange DriverContext collectDriverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); ValuesOperator.ValuesOperatorFactory valuesOperatorFactory = new ValuesOperator.ValuesOperatorFactory(0, new PlanNodeId("values"), buildPages.build()); - LocalExchangeSinkOperator.LocalExchangeSinkOperatorFactory sinkOperatorFactory = new LocalExchangeSinkOperator.LocalExchangeSinkOperatorFactory(localExchangeFactory, 1, new PlanNodeId("sink"), localExchangeSinkFactoryId, Function.identity()); + LocalExchangeSinkOperator.LocalExchangeSinkOperatorFactory sinkOperatorFactory = new LocalExchangeSinkOperator.LocalExchangeSinkOperatorFactory(localExchange.createSinkFactory(), 1, new PlanNodeId("sink"), Function.identity()); Driver sourceDriver = Driver.createDriver(collectDriverContext, valuesOperatorFactory.createOperator(collectDriverContext), sinkOperatorFactory.createOperator(collectDriverContext)); valuesOperatorFactory.noMoreOperators(); sinkOperatorFactory.noMoreOperators(); + sinkOperatorFactory.localPlannerComplete(); while (!sourceDriver.isFinished()) { sourceDriver.processUntilBlocked(); } // build side operator factories - LocalExchangeSourceOperatorFactory sourceOperatorFactory = new LocalExchangeSourceOperatorFactory(0, new PlanNodeId("source"), localExchangeFactory); + LocalExchangeSourceOperatorFactory sourceOperatorFactory = new LocalExchangeSourceOperatorFactory(0, new PlanNodeId("source"), localExchange); JoinBridgeManager lookupSourceFactoryManager = JoinBridgeManager.lookupAllAtOnce(new PartitionedLookupSourceFactory( buildPages.getTypes(), rangeList(buildPages.getTypes().size()).stream() @@ -208,7 +204,7 @@ public static void buildLookupSource(ExecutorService executor, BuildSideSetup bu { requireNonNull(buildSideSetup, "buildSideSetup is null"); - LookupSourceFactory lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager().getJoinBridge(Lifespan.taskWide()); + LookupSourceFactory lookupSourceFactory = buildSideSetup.getLookupSourceFactoryManager().getJoinBridge(); Future lookupSourceProvider = lookupSourceFactory.createLookupSourceProvider(); List buildDrivers = buildSideSetup.getBuildDrivers(); diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java index 6debffb7070d..1f8302fd8430 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestHashJoinOperator.java @@ -24,7 +24,6 @@ import io.trino.ExceededMemoryLimitException; import io.trino.RowPagesBuilder; import io.trino.connector.CatalogServiceProvider; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap; import io.trino.execution.StageId; import io.trino.execution.TaskId; @@ -524,7 +523,7 @@ private void innerJoinWithSpill(boolean probeHashEnabled, List whenSp List buildDrivers = buildSideSetup.getBuildDrivers(); int buildOperatorCount = buildDrivers.size(); checkState(buildOperatorCount == whenSpill.size()); - LookupSourceFactory lookupSourceFactory = lookupSourceFactoryManager.getJoinBridge(Lifespan.taskWide()); + LookupSourceFactory lookupSourceFactory = lookupSourceFactoryManager.getJoinBridge(); try (Operator joinOperator = joinOperatorFactory.createOperator(joinDriverContext)) { // build lookup source @@ -674,7 +673,7 @@ public void testBuildGracefulSpill() instantiateBuildDrivers(buildSideSetup, taskContext); JoinBridgeManager lookupSourceFactoryManager = buildSideSetup.getLookupSourceFactoryManager(); - PartitionedLookupSourceFactory lookupSourceFactory = lookupSourceFactoryManager.getJoinBridge(Lifespan.taskWide()); + PartitionedLookupSourceFactory lookupSourceFactory = lookupSourceFactoryManager.getJoinBridge(); // finish probe before any build partition is spilled lookupSourceFactory.finishProbeOperator(OptionalInt.of(1)); diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopBuildOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopBuildOperator.java index 8df1ca997555..1fe1a0c5076d 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopBuildOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopBuildOperator.java @@ -14,9 +14,7 @@ package io.trino.operator.join; import com.google.common.collect.ImmutableList; -import io.trino.execution.Lifespan; import io.trino.operator.DriverContext; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.TaskContext; import io.trino.operator.join.NestedLoopBuildOperator.NestedLoopBuildOperatorFactory; import io.trino.operator.project.PageProcessor; @@ -70,14 +68,12 @@ public void testNestedLoopBuild() List buildTypes = ImmutableList.of(BIGINT); JoinBridgeManager nestedLoopJoinBridgeManager = new JoinBridgeManager<>( false, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - lifespan -> new NestedLoopJoinPagesSupplier(), + new NestedLoopJoinPagesSupplier(), buildTypes); NestedLoopBuildOperatorFactory nestedLoopBuildOperatorFactory = new NestedLoopBuildOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinBridgeManager); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); NestedLoopBuildOperator nestedLoopBuildOperator = (NestedLoopBuildOperator) nestedLoopBuildOperatorFactory.createOperator(driverContext); - NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(Lifespan.taskWide()); + NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(); assertFalse(nestedLoopJoinBridge.getPagesFuture().isDone()); @@ -107,14 +103,12 @@ public void testNestedLoopBuildNoBlock() List buildTypes = ImmutableList.of(); JoinBridgeManager nestedLoopJoinBridgeManager = new JoinBridgeManager<>( false, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - lifespan -> new NestedLoopJoinPagesSupplier(), + new NestedLoopJoinPagesSupplier(), buildTypes); NestedLoopBuildOperatorFactory nestedLoopBuildOperatorFactory = new NestedLoopBuildOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinBridgeManager); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); NestedLoopBuildOperator nestedLoopBuildOperator = (NestedLoopBuildOperator) nestedLoopBuildOperatorFactory.createOperator(driverContext); - NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(Lifespan.taskWide()); + NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(); assertFalse(nestedLoopJoinBridge.getPagesFuture().isDone()); @@ -143,14 +137,12 @@ public void testNestedLoopNoBlocksMaxSizeLimit() List buildTypes = ImmutableList.of(); JoinBridgeManager nestedLoopJoinBridgeManager = new JoinBridgeManager<>( false, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - lifespan -> new NestedLoopJoinPagesSupplier(), + new NestedLoopJoinPagesSupplier(), buildTypes); NestedLoopBuildOperatorFactory nestedLoopBuildOperatorFactory = new NestedLoopBuildOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinBridgeManager); DriverContext driverContext = taskContext.addPipelineContext(0, true, true, false).addDriverContext(); NestedLoopBuildOperator nestedLoopBuildOperator = (NestedLoopBuildOperator) nestedLoopBuildOperatorFactory.createOperator(driverContext); - NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(Lifespan.taskWide()); + NestedLoopJoinBridge nestedLoopJoinBridge = nestedLoopJoinBridgeManager.getJoinBridge(); assertFalse(nestedLoopJoinBridge.getPagesFuture().isDone()); diff --git a/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopJoinOperator.java b/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopJoinOperator.java index 8dbce0e28ef6..c6132dfe3737 100644 --- a/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopJoinOperator.java +++ b/core/trino-main/src/test/java/io/trino/operator/join/TestNestedLoopJoinOperator.java @@ -18,7 +18,6 @@ import io.trino.operator.Driver; import io.trino.operator.DriverContext; import io.trino.operator.Operator; -import io.trino.operator.PipelineExecutionStrategy; import io.trino.operator.TaskContext; import io.trino.operator.join.NestedLoopBuildOperator.NestedLoopBuildOperatorFactory; import io.trino.operator.join.NestedLoopJoinOperator.NestedLoopJoinOperatorFactory; @@ -512,9 +511,7 @@ private static NestedLoopJoinOperatorFactory newJoinOperatorFactoryWithCompleted JoinBridgeManager nestedLoopJoinBridgeManager = new JoinBridgeManager<>( false, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - PipelineExecutionStrategy.UNGROUPED_EXECUTION, - lifespan -> new NestedLoopJoinPagesSupplier(), + new NestedLoopJoinPagesSupplier(), buildPages.getTypes()); NestedLoopBuildOperatorFactory nestedLoopBuildOperatorFactory = new NestedLoopBuildOperatorFactory(1, new PlanNodeId("test"), nestedLoopJoinBridgeManager); NestedLoopJoinOperatorFactory joinOperatorFactory = new NestedLoopJoinOperatorFactory(3, new PlanNodeId("test"), nestedLoopJoinBridgeManager, probeChannels, buildChannels); diff --git a/core/trino-main/src/test/java/io/trino/operator/scalar/FunctionAssertions.java b/core/trino-main/src/test/java/io/trino/operator/scalar/FunctionAssertions.java index 992f14de89b2..73e95cfd639f 100644 --- a/core/trino-main/src/test/java/io/trino/operator/scalar/FunctionAssertions.java +++ b/core/trino-main/src/test/java/io/trino/operator/scalar/FunctionAssertions.java @@ -25,7 +25,6 @@ import io.trino.FeaturesConfig; import io.trino.Session; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.FunctionBundle; import io.trino.metadata.FunctionManager; import io.trino.metadata.Metadata; @@ -973,12 +972,12 @@ public ConnectorPageSource createPageSource(Session session, Split split, TableH private static Split createRecordSetSplit() { - return new Split(new CatalogName("test"), new TestSplit(true), Lifespan.taskWide()); + return new Split(new CatalogName("test"), new TestSplit(true)); } private static Split createNormalSplit() { - return new Split(new CatalogName("test"), new TestSplit(false), Lifespan.taskWide()); + return new Split(new CatalogName("test"), new TestSplit(false)); } private static RowType createTestRowType(int numberOfFields) diff --git a/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java b/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java index c93e6e9d1b85..f57048931695 100644 --- a/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java +++ b/core/trino-main/src/test/java/io/trino/server/TestDynamicFilterService.java @@ -57,7 +57,6 @@ import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static io.trino.metadata.MetadataManager.createTestMetadataManager; -import static io.trino.operator.StageExecutionDescriptor.ungroupedExecution; import static io.trino.server.DynamicFilterService.DynamicFilterDomainStats; import static io.trino.server.DynamicFilterService.DynamicFiltersStats; import static io.trino.server.DynamicFilterService.getOutboundDynamicFilters; @@ -902,7 +901,6 @@ private static PlanFragment createPlan( stagePartitioning, ImmutableList.of(tableScanNodeId), new PartitioningScheme(Partitioning.create(SINGLE_DISTRIBUTION, ImmutableList.of()), ImmutableList.of(symbol)), - ungroupedExecution(), StatsAndCosts.empty(), Optional.empty()); } 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 8ea37414db9b..1307cdce9665 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 @@ -33,7 +33,6 @@ import io.trino.connector.CatalogName; import io.trino.execution.DynamicFilterConfig; import io.trino.execution.DynamicFiltersCollector.VersionedDynamicFilterDomains; -import io.trino.execution.Lifespan; import io.trino.execution.NodeTaskMap; import io.trino.execution.QueryManagerConfig; import io.trino.execution.RemoteTask; @@ -183,14 +182,10 @@ public void testRegular() testingTaskResource.setInitialTaskInfo(remoteTask.getTaskInfo()); remoteTask.start(); - Lifespan lifespan = Lifespan.driverGroup(3); - remoteTask.addSplits(ImmutableMultimap.of(TABLE_SCAN_NODE_ID, new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), lifespan))); + remoteTask.addSplits(ImmutableMultimap.of(TABLE_SCAN_NODE_ID, new Split(new CatalogName("test"), TestingSplit.createLocalSplit()))); poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID) != null); poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID).getSplits().size() == 1); - remoteTask.noMoreSplits(TABLE_SCAN_NODE_ID, lifespan); - poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID).getNoMoreSplitsForLifespan().size() == 1); - remoteTask.noMoreSplits(TABLE_SCAN_NODE_ID); poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID).isNoMoreSplits()); @@ -413,8 +408,7 @@ private void runTest(FailureScenario failureScenario) private void addSplit(RemoteTask remoteTask, TestingTaskResource testingTaskResource, int expectedSplitsCount) throws InterruptedException { - Lifespan lifespan = Lifespan.driverGroup(3); - remoteTask.addSplits(ImmutableMultimap.of(TABLE_SCAN_NODE_ID, new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), lifespan))); + remoteTask.addSplits(ImmutableMultimap.of(TABLE_SCAN_NODE_ID, new Split(new CatalogName("test"), TestingSplit.createLocalSplit()))); // wait for splits to be received by remote task poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID) != null); poll(() -> testingTaskResource.getTaskSplitAssignment(TABLE_SCAN_NODE_ID).getSplits().size() == expectedSplitsCount); @@ -623,7 +617,7 @@ public synchronized SplitAssignment getTaskSplitAssignment(PlanNodeId planNodeId if (assignment == null) { return null; } - return new SplitAssignment(assignment.getPlanNodeId(), assignment.getSplits(), assignment.getNoMoreSplitsForLifespan(), assignment.isNoMoreSplits()); + return new SplitAssignment(assignment.getPlanNodeId(), assignment.getSplits(), assignment.isNoMoreSplits()); } @GET @@ -775,7 +769,6 @@ private TaskStatus buildTaskStatus() taskState, initialTaskStatus.getSelf(), "fake", - ImmutableSet.of(), initialTaskStatus.getFailures(), initialTaskStatus.getQueuedPartitionedDrivers(), initialTaskStatus.getRunningPartitionedDrivers(), diff --git a/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java b/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java index 74045db3d736..77a5c4c57371 100644 --- a/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java +++ b/core/trino-main/src/test/java/io/trino/split/MockSplitSource.java @@ -18,10 +18,8 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.metadata.Split; import io.trino.spi.HostAddress; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import javax.annotation.concurrent.NotThreadSafe; @@ -33,7 +31,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.MoreExecutors.directExecutor; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.split.MockSplitSource.Action.DO_NOTHING; import static io.trino.split.MockSplitSource.Action.FINISH; @@ -41,7 +38,7 @@ public class MockSplitSource implements SplitSource { - private static final Split SPLIT = new Split(new CatalogName("test"), new MockConnectorSplit(), Lifespan.taskWide()); + private static final Split SPLIT = new Split(new CatalogName("test"), new MockConnectorSplit()); private static final SettableFuture> COMPLETED_FUTURE = SettableFuture.create(); static { @@ -115,13 +112,8 @@ private void doGetNextBatch() } @Override - public ListenableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, Lifespan lifespan, int maxSize) + public ListenableFuture getNextBatch(int maxSize) { - if (partitionHandle != NOT_PARTITIONED) { - throw new UnsupportedOperationException(); - } - checkArgument(Lifespan.taskWide().equals(lifespan)); - checkState(nextBatchFuture.isDone(), "concurrent getNextBatch invocation"); nextBatchFuture = SettableFuture.create(); nextBatchMaxSize = maxSize; diff --git a/core/trino-main/src/test/java/io/trino/split/TestBufferingSplitSource.java b/core/trino-main/src/test/java/io/trino/split/TestBufferingSplitSource.java index b5092b2b3447..47a634b7d78f 100644 --- a/core/trino-main/src/test/java/io/trino/split/TestBufferingSplitSource.java +++ b/core/trino-main/src/test/java/io/trino/split/TestBufferingSplitSource.java @@ -15,7 +15,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import io.trino.execution.Lifespan; import io.trino.split.SplitSource.SplitBatch; import org.testng.annotations.Test; @@ -23,7 +22,6 @@ import static com.google.common.util.concurrent.MoreExecutors.directExecutor; import static io.airlift.concurrent.MoreFutures.tryGetFutureValue; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.split.MockSplitSource.Action.FAIL; import static io.trino.split.MockSplitSource.Action.FINISH; import static java.util.Objects.requireNonNull; @@ -238,7 +236,7 @@ private static T requireFutureValue(Future future) private static ListenableFuture getNextBatch(SplitSource splitSource, int maxSize) { - ListenableFuture future = splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), maxSize); + ListenableFuture future = splitSource.getNextBatch(maxSize); return Futures.transform(future, NextBatchResult::new, directExecutor()); } diff --git a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java index 66aec81395fe..e7a1b36d4d31 100644 --- a/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java +++ b/core/trino-main/src/test/java/io/trino/sql/analyzer/TestFeaturesConfig.java @@ -36,9 +36,6 @@ public class TestFeaturesConfig public void testDefaults() { assertRecordedDefaults(recordDefaults(FeaturesConfig.class) - .setGroupedExecutionEnabled(false) - .setDynamicScheduleForGroupedExecutionEnabled(false) - .setConcurrentLifespansPerTask(0) .setRedistributeWrites(true) .setScaleWriters(true) .setWriterMinSize(DataSize.of(32, MEGABYTE)) @@ -73,9 +70,6 @@ public void testDefaults() public void testExplicitPropertyMappings() { Map properties = ImmutableMap.builder() - .put("grouped-execution-enabled", "true") - .put("dynamic-schedule-for-grouped-execution", "true") - .put("concurrent-lifespans-per-task", "1") .put("redistribute-writes", "false") .put("scale-writers", "false") .put("writer-min-size", "42GB") @@ -107,9 +101,6 @@ public void testExplicitPropertyMappings() .buildOrThrow(); FeaturesConfig expected = new FeaturesConfig() - .setGroupedExecutionEnabled(true) - .setDynamicScheduleForGroupedExecutionEnabled(true) - .setConcurrentLifespansPerTask(1) .setRedistributeWrites(false) .setScaleWriters(false) .setWriterMinSize(DataSize.of(42, GIGABYTE)) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java index 6afb241b0d26..826af8e83770 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/planprinter/TestJsonRepresentation.java @@ -207,7 +207,6 @@ private void assertJsonRepresentation(Function sourceNode PlanPrinter planPrinter = new PlanPrinter( sourceNodeSupplier.apply(planBuilder), planBuilder.getTypes(), - Optional.empty(), scanNode -> TABLE_INFO, ImmutableMap.of(), valuePrinter, diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNodePartitioningProvider.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNodePartitioningProvider.java index 5bcaa5a92af6..35c921d01f5c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNodePartitioningProvider.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorNodePartitioningProvider.java @@ -18,9 +18,6 @@ import java.util.List; import java.util.function.ToIntFunction; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; -import static java.util.Collections.singletonList; - public interface ConnectorNodePartitioningProvider { // TODO: Use ConnectorPartitionHandle (instead of int) to represent individual buckets. @@ -31,10 +28,13 @@ public interface ConnectorNodePartitioningProvider *

* This method must be implemented for connectors that support addressable split discovery. * The partitions return here will be used as address for the purpose of split discovery. + * + * @deprecated The method is not used. Implementations can be simply removed */ + @Deprecated default List listPartitionHandles(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle) { - return singletonList(NOT_PARTITIONED); + throw new UnsupportedOperationException(); } ConnectorBucketNodeMap getBucketNodeMap(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPartitionHandle.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPartitionHandle.java index 87540c4d7664..9dd4dc1bb02a 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPartitionHandle.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorPartitionHandle.java @@ -14,6 +14,7 @@ package io.trino.spi.connector; @SuppressWarnings("ClassMayBeInterface") +@Deprecated public abstract class ConnectorPartitionHandle { @Override diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitManager.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitManager.java index 27dcc4d9e332..860720e32c25 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitManager.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitManager.java @@ -13,8 +13,11 @@ */ package io.trino.spi.connector; +import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; + public interface ConnectorSplitManager { + @Deprecated default ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, @@ -26,6 +29,17 @@ default ConnectorSplitSource getSplits( throw new UnsupportedOperationException(); } + default ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle table, + DynamicFilter dynamicFilter, + Constraint constraint) + { + return getSplits(transaction, session, table, UNGROUPED_SCHEDULING, dynamicFilter, constraint); + } + + @Deprecated enum SplitSchedulingStrategy { UNGROUPED_SCHEDULING, diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java index d585644d6420..f19b7498bec6 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/ConnectorSplitSource.java @@ -18,12 +18,22 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; +import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.Objects.requireNonNull; public interface ConnectorSplitSource extends Closeable { - CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize); + @Deprecated + default CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + { + throw new UnsupportedOperationException(); + } + + default CompletableFuture getNextBatch(int maxSize) + { + return getNextBatch(NOT_PARTITIONED, maxSize); + } @Override void close(); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java b/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java index 0979a6319459..a7b8f61fb82d 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/FixedSplitSource.java @@ -17,7 +17,6 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.stream.Collectors.toUnmodifiableList; @@ -50,12 +49,8 @@ private FixedSplitSource(Iterable splits, Optional getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { - if (!partitionHandle.equals(NOT_PARTITIONED)) { - throw new IllegalArgumentException("partitionHandle must be NOT_PARTITIONED"); - } - int remainingSplits = splits.size() - offset; int size = Math.min(remainingSplits, maxSize); List results = splits.subList(offset, offset + size); diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/NotPartitionedPartitionHandle.java b/core/trino-spi/src/main/java/io/trino/spi/connector/NotPartitionedPartitionHandle.java index a6a5d246ad49..5f4c7186ff8c 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/NotPartitionedPartitionHandle.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/NotPartitionedPartitionHandle.java @@ -13,6 +13,7 @@ */ package io.trino.spi.connector; +@Deprecated public final class NotPartitionedPartitionHandle extends ConnectorPartitionHandle { diff --git a/core/trino-spi/src/test/java/io/trino/spi/TestSpiBackwardCompatibility.java b/core/trino-spi/src/test/java/io/trino/spi/TestSpiBackwardCompatibility.java index a6bd9db1063d..7ba3992ed0df 100644 --- a/core/trino-spi/src/test/java/io/trino/spi/TestSpiBackwardCompatibility.java +++ b/core/trino-spi/src/test/java/io/trino/spi/TestSpiBackwardCompatibility.java @@ -72,6 +72,8 @@ public class TestSpiBackwardCompatibility .put("386", "Method: public default boolean io.trino.spi.connector.ConnectorMetadata.isSupportedVersionType(io.trino.spi.connector.ConnectorSession,io.trino.spi.connector.SchemaTableName,io.trino.spi.connector.PointerType,io.trino.spi.type.Type)") .put("386", "Method: public static io.trino.spi.ptf.TableArgumentSpecification$Builder io.trino.spi.ptf.TableArgumentSpecification.builder(java.lang.String)") .put("387", "Constructor: public io.trino.spi.eventlistener.QueryContext(java.lang.String,java.util.Optional,java.util.Set,java.util.Optional,java.util.Optional,java.util.Optional,java.util.Optional,java.util.Set,java.util.Set,java.util.Optional,java.util.Optional,java.util.Optional,java.util.Optional,java.util.Map,io.trino.spi.session.ResourceEstimates,java.lang.String,java.lang.String,java.lang.String,java.util.Optional)") + .put("388", "Method: public abstract java.util.concurrent.CompletableFuture io.trino.spi.connector.ConnectorSplitSource.getNextBatch(io.trino.spi.connector.ConnectorPartitionHandle,int)") + .put("388", "Method: public java.util.concurrent.CompletableFuture io.trino.spi.connector.FixedSplitSource.getNextBatch(io.trino.spi.connector.ConnectorPartitionHandle,int)") .build(); @Test diff --git a/docs/src/main/sphinx/admin/dynamic-filtering.rst b/docs/src/main/sphinx/admin/dynamic-filtering.rst index 0f4ea5493e7b..4477dd7cddbf 100644 --- a/docs/src/main/sphinx/admin/dynamic-filtering.rst +++ b/docs/src/main/sphinx/admin/dynamic-filtering.rst @@ -95,7 +95,6 @@ down to the connector in the query plan. Fragment 1 [SOURCE] Output layout: [count_3] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Aggregate(PARTIAL) │ Layout: [count_3:bigint] │ count_3 := count(*) @@ -118,7 +117,6 @@ down to the connector in the query plan. Fragment 2 [SOURCE] Output layout: [d_date_sk, $hashvalue_6] Output partitioning: BROADCAST [] - Stage Execution Strategy: UNGROUPED_EXECUTION ScanFilterProject[table = hive:default:date_dim, grouped = false, filterPredicate = ((""d_following_holiday"" = CAST('Y' AS char(1))) AND (""d_year"" = 2000))] Layout: [d_date_sk:bigint, $hashvalue_6:bigint] Estimates: {rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B}/{rows: 0 (0B), cpu: 0, memory: 0B, network: 0B} diff --git a/docs/src/main/sphinx/optimizer/pushdown.rst b/docs/src/main/sphinx/optimizer/pushdown.rst index 1cda33ce9a2b..447281295a3d 100644 --- a/docs/src/main/sphinx/optimizer/pushdown.rst +++ b/docs/src/main/sphinx/optimizer/pushdown.rst @@ -102,7 +102,6 @@ operator. This shows you that the pushdown was successful. Fragment 0 [SINGLE] Output layout: [regionkey_0, _generated_1] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Output[regionkey, _col1] │ Layout: [regionkey_0:bigint, _generated_1:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?} @@ -114,7 +113,6 @@ operator. This shows you that the pushdown was successful. Fragment 1 [SOURCE] Output layout: [regionkey_0, _generated_1] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION TableScan[postgresql:tpch.nation tpch.nation columns=[regionkey:bigint:int8, count(*):_generated_1:bigint:bigint] groupingSets=[[regionkey:bigint:int8]], gro Layout: [regionkey_0:bigint, _generated_1:bigint] Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} @@ -136,7 +134,6 @@ performed, and instead Trino performs the aggregate processing. Fragment 0 [SINGLE] Output layout: [regionkey, count] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Output[regionkey, _col1] │ Layout: [regionkey:bigint, count:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} @@ -147,7 +144,6 @@ performed, and instead Trino performs the aggregate processing. Fragment 1 [HASH] Output layout: [regionkey, count] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Aggregate(FINAL)[regionkey] │ Layout: [regionkey:bigint, count:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} @@ -161,7 +157,6 @@ performed, and instead Trino performs the aggregate processing. Fragment 2 [SOURCE] Output layout: [regionkey, count_0, $hashvalue_2] Output partitioning: HASH [regionkey][$hashvalue_2] - Stage Execution Strategy: UNGROUPED_EXECUTION Project[] │ Layout: [regionkey:bigint, count_0:bigint, $hashvalue_2:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} @@ -220,7 +215,6 @@ result of the successful join push down. Fragment 0 [SINGLE] Output layout: [custkey, orderkey] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Output[custkey, orderkey] │ Layout: [custkey:bigint, orderkey:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: ?} @@ -230,7 +224,6 @@ result of the successful join push down. Fragment 1 [SOURCE] Output layout: [orderkey, custkey] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION TableScan[postgres:Query[SELECT l."orderkey" AS "orderkey_0", l."custkey" AS "custkey_1", r."custkey" AS "custkey_2" FROM (SELECT "orderkey", "custkey" FROM "tpch"."orders") l INNER JOIN (SELECT "custkey" FROM "tpch"."customer") r O Layout: [orderkey:bigint, custkey:bigint] Estimates: {rows: ? (?), cpu: ?, memory: 0B, network: 0B} diff --git a/docs/src/main/sphinx/sql/explain.rst b/docs/src/main/sphinx/sql/explain.rst index e76c3ef807b7..8bb5c493cba2 100644 --- a/docs/src/main/sphinx/sql/explain.rst +++ b/docs/src/main/sphinx/sql/explain.rst @@ -362,7 +362,6 @@ Distributed plan:: Fragment 0 [SINGLE] Output layout: [regionkey, count] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Output[regionkey, _col1] │ Layout: [regionkey:bigint, count:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} @@ -373,7 +372,6 @@ Distributed plan:: Fragment 1 [HASH] Output layout: [regionkey, count] Output partitioning: SINGLE [] - Stage Execution Strategy: UNGROUPED_EXECUTION Aggregate(FINAL)[regionkey] │ Layout: [regionkey:bigint, count:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} @@ -387,7 +385,6 @@ Distributed plan:: Fragment 2 [SOURCE] Output layout: [regionkey, count_8, $hashvalue_10] Output partitioning: HASH [regionkey][$hashvalue_10] - Stage Execution Strategy: UNGROUPED_EXECUTION Project[] │ Layout: [regionkey:bigint, count_8:bigint, $hashvalue_10:bigint] │ Estimates: {rows: ? (?), cpu: ?, memory: ?, network: ?} diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java index 73211cdc46e6..fefca61158b0 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitManager.java @@ -39,6 +39,19 @@ public ClassLoaderSafeConnectorSplitManager(@ForClassLoaderSafe ConnectorSplitMa this.classLoader = requireNonNull(classLoader, "classLoader is null"); } + @Override + public ConnectorSplitSource getSplits( + ConnectorTransactionHandle transaction, + ConnectorSession session, + ConnectorTableHandle table, + DynamicFilter dynamicFilter, + Constraint constraint) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getSplits(transaction, session, table, dynamicFilter, constraint); + } + } + @Override public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java index ca388c69ccfa..66f16fbb38a2 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeConnectorSplitSource.java @@ -38,6 +38,7 @@ public ClassLoaderSafeConnectorSplitSource(@ForClassLoaderSafe ConnectorSplitSou this.classLoader = requireNonNull(classLoader, "classLoader is null"); } + @Deprecated @Override public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) { @@ -46,6 +47,14 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan } } + @Override + public CompletableFuture getNextBatch(int maxSize) + { + try (ThreadContextClassLoader ignored = new ThreadContextClassLoader(classLoader)) { + return delegate.getNextBatch(maxSize); + } + } + @Override public Optional> getTableExecuteSplitsInfo() { diff --git a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java index 01b1c3993ea9..8126699dbeaf 100644 --- a/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java +++ b/lib/trino-plugin-toolkit/src/main/java/io/trino/plugin/base/classloader/ClassLoaderSafeNodePartitioningProvider.java @@ -57,6 +57,7 @@ public BucketFunction getBucketFunction( } } + @Deprecated @Override public List listPartitionHandles(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle) { diff --git a/plugin/trino-accumulo/src/main/java/io/trino/plugin/accumulo/AccumuloSplitManager.java b/plugin/trino-accumulo/src/main/java/io/trino/plugin/accumulo/AccumuloSplitManager.java index daebe5da99ca..d0b0b5172317 100644 --- a/plugin/trino-accumulo/src/main/java/io/trino/plugin/accumulo/AccumuloSplitManager.java +++ b/plugin/trino-accumulo/src/main/java/io/trino/plugin/accumulo/AccumuloSplitManager.java @@ -58,7 +58,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-atop/src/main/java/io/trino/plugin/atop/AtopSplitManager.java b/plugin/trino-atop/src/main/java/io/trino/plugin/atop/AtopSplitManager.java index 091ae5caa73a..4ebdd9c4fedb 100644 --- a/plugin/trino-atop/src/main/java/io/trino/plugin/atop/AtopSplitManager.java +++ b/plugin/trino-atop/src/main/java/io/trino/plugin/atop/AtopSplitManager.java @@ -61,7 +61,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSplitManager.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSplitManager.java index 259cf984f953..0b6864a5d420 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSplitManager.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcSplitManager.java @@ -41,7 +41,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java index 37ca8f9be138..cc5f31a25d67 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcRecordSetProvider.java @@ -43,7 +43,6 @@ import static io.airlift.concurrent.MoreFutures.getFutureValue; import static io.airlift.slice.Slices.utf8Slice; import static io.airlift.testing.Closeables.closeAll; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createVarcharType; @@ -203,7 +202,7 @@ private RecordCursor getCursor(JdbcTableHandle jdbcTableHandle, List getColumnHandles(ConnectorSession session, JdbcTableHandle table) diff --git a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitManager.java b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitManager.java index e984af7af095..bcf8d02342a0 100644 --- a/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitManager.java +++ b/plugin/trino-bigquery/src/main/java/io/trino/plugin/bigquery/BigQuerySplitManager.java @@ -90,11 +90,10 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { - log.debug("getSplits(transaction=%s, session=%s, table=%s, splitSchedulingStrategy=%s)", transaction, session, table, splitSchedulingStrategy); + log.debug("getSplits(transaction=%s, session=%s, table=%s)", transaction, session, table); BigQueryTableHandle bigQueryTableHandle = (BigQueryTableHandle) table; TableId remoteTableId = bigQueryTableHandle.getRemoteTableName().toTableId(); diff --git a/plugin/trino-blackhole/src/main/java/io/trino/plugin/blackhole/BlackHoleSplitManager.java b/plugin/trino-blackhole/src/main/java/io/trino/plugin/blackhole/BlackHoleSplitManager.java index f3b25d590d8b..3ccef0527b6b 100644 --- a/plugin/trino-blackhole/src/main/java/io/trino/plugin/blackhole/BlackHoleSplitManager.java +++ b/plugin/trino-blackhole/src/main/java/io/trino/plugin/blackhole/BlackHoleSplitManager.java @@ -32,7 +32,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraSplitManager.java b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraSplitManager.java index 75bbf8a05490..7fe83552ed15 100644 --- a/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraSplitManager.java +++ b/plugin/trino-cassandra/src/main/java/io/trino/plugin/cassandra/CassandraSplitManager.java @@ -76,7 +76,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnector.java b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnector.java index a5210f4207d1..b28042db4c07 100644 --- a/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnector.java +++ b/plugin/trino-cassandra/src/test/java/io/trino/plugin/cassandra/TestCassandraConnector.java @@ -64,8 +64,6 @@ import static io.trino.plugin.cassandra.CassandraTestingUtils.TABLE_TUPLE_TYPE; import static io.trino.plugin.cassandra.CassandraTestingUtils.TABLE_USER_DEFINED_TYPE; import static io.trino.plugin.cassandra.CassandraTestingUtils.createTestTables; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.BooleanType.BOOLEAN; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; @@ -188,7 +186,7 @@ public void testGetRecords() tableHandle = metadata.applyFilter(SESSION, tableHandle, Constraint.alwaysTrue()).get().getHandle(); - List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue())); + List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue())); long rowNumber = 0; for (ConnectorSplit split : splits) { @@ -260,7 +258,7 @@ public void testGetTupleType() ConnectorTransactionHandle transaction = CassandraTransactionHandle.INSTANCE; - List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue())); + List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue())); long rowNumber = 0; for (ConnectorSplit split : splits) { @@ -311,7 +309,7 @@ public void testGetUserDefinedType() tableHandle = metadata.applyFilter(SESSION, tableHandle, Constraint.alwaysTrue()).get().getHandle(); - List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue())); + List splits = getAllSplits(splitManager.getSplits(transaction, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue())); long rowNumber = 0; for (ConnectorSplit split : splits) { @@ -433,7 +431,7 @@ private static List getAllSplits(ConnectorSplitSource splitSourc { ImmutableList.Builder splits = ImmutableList.builder(); while (!splitSource.isFinished()) { - splits.addAll(getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, 1000)).getSplits()); + splits.addAll(getFutureValue(splitSource.getNextBatch(1000)).getSplits()); } return splits.build(); } diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java index 605a961fe00a..a119a2494884 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitManager.java @@ -94,7 +94,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle handle, - ConnectorSplitManager.SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java index ea9ab667a3c7..38ae0b3fb0ca 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeSplitSource.java @@ -23,7 +23,6 @@ import io.trino.plugin.hive.util.AsyncQueue; import io.trino.plugin.hive.util.ThrottledAsyncQueue; import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.DynamicFilter; @@ -98,7 +97,7 @@ public DeltaLakeSplitSource( } @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { long timeLeft = dynamicFilteringWaitTimeoutMillis - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); if (dynamicFilter.isAwaitable() && timeLeft > 0) { diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java index c6a4fa58d5cf..773ddf8640d2 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeDynamicFiltering.java @@ -18,7 +18,6 @@ import com.google.common.collect.ImmutableSet; import io.trino.Session; import io.trino.execution.DynamicFilterConfig; -import io.trino.execution.Lifespan; import io.trino.execution.QueryStats; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.Split; @@ -56,9 +55,7 @@ import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; import static io.trino.plugin.deltalake.DeltaLakeDockerizedMinioDataLake.createDockerizedMinioDataLakeForDeltaLake; import static io.trino.plugin.deltalake.DeltaLakeQueryRunner.DELTA_CATALOG; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.tpch.TpchTable.LINE_ITEM; import static io.trino.tpch.TpchTable.ORDERS; @@ -132,10 +129,10 @@ public void testIncompleteDynamicFilterTimeout() Optional tableHandle = runner.getMetadata().getTableHandle(session, tableName); assertTrue(tableHandle.isPresent()); SplitSource splitSource = runner.getSplitManager() - .getSplits(session, tableHandle.get(), UNGROUPED_SCHEDULING, new IncompleteDynamicFilter(), alwaysTrue()); + .getSplits(session, tableHandle.get(), new IncompleteDynamicFilter(), alwaysTrue()); List splits = new ArrayList<>(); while (!splitSource.isFinished()) { - splits.addAll(splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), 1000).get().getSplits()); + splits.addAll(splitSource.getNextBatch(1000).get().getSplits()); } splitSource.close(); assertFalse(splits.isEmpty()); diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java index 8b5b2407b21d..cd91249c100a 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeSplitManager.java @@ -32,7 +32,6 @@ import io.trino.spi.SplitWeight; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; -import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.Constraint; import io.trino.spi.connector.DynamicFilter; @@ -49,7 +48,6 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.testing.assertions.Assert.assertEquals; public class TestDeltaLakeSplitManager @@ -189,12 +187,11 @@ private List getSplits(DeltaLakeSplitManager splitManager, Delta new HiveTransactionHandle(false), testingConnectorSessionWithConfig(deltaLakeConfig), tableHandle, - ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); ImmutableList.Builder splits = ImmutableList.builder(); while (!splitSource.isFinished()) { - List nextBatch = splitSource.getNextBatch(NOT_PARTITIONED, 10).get().getSplits(); + List nextBatch = splitSource.getNextBatch(10).get().getSplits(); splits.addAll( nextBatch.stream() .map(split -> (DeltaLakeSplit) split) diff --git a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java index 2315a210b23c..18fce4cba83e 100644 --- a/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java +++ b/plugin/trino-elasticsearch/src/main/java/io/trino/plugin/elasticsearch/ElasticsearchSplitManager.java @@ -49,7 +49,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-example-http/src/main/java/io/trino/plugin/example/ExampleSplitManager.java b/plugin/trino-example-http/src/main/java/io/trino/plugin/example/ExampleSplitManager.java index 71b7c9a18244..e41f44e0c32c 100644 --- a/plugin/trino-example-http/src/main/java/io/trino/plugin/example/ExampleSplitManager.java +++ b/plugin/trino-example-http/src/main/java/io/trino/plugin/example/ExampleSplitManager.java @@ -49,7 +49,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle connectorTableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-google-sheets/src/main/java/io/trino/plugin/google/sheets/SheetsSplitManager.java b/plugin/trino-google-sheets/src/main/java/io/trino/plugin/google/sheets/SheetsSplitManager.java index fe0ecb3a105e..5f763aa16d9d 100644 --- a/plugin/trino-google-sheets/src/main/java/io/trino/plugin/google/sheets/SheetsSplitManager.java +++ b/plugin/trino-google-sheets/src/main/java/io/trino/plugin/google/sheets/SheetsSplitManager.java @@ -49,7 +49,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle connectorTableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveNodePartitioningProvider.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveNodePartitioningProvider.java index 52ad45a69093..d47d8e8f02e1 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveNodePartitioningProvider.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveNodePartitioningProvider.java @@ -19,7 +19,6 @@ import io.trino.spi.connector.BucketFunction; import io.trino.spi.connector.ConnectorBucketNodeMap; import io.trino.spi.connector.ConnectorNodePartitioningProvider; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; @@ -35,7 +34,6 @@ import java.util.Collections; import java.util.List; import java.util.function.ToIntFunction; -import java.util.stream.IntStream; import java.util.stream.Stream; import static com.google.common.collect.ImmutableList.toImmutableList; @@ -125,12 +123,4 @@ public ToIntFunction getSplitBucketFunction( return value -> ((HiveSplit) value).getReadBucketNumber() .orElseThrow(() -> new IllegalArgumentException("Bucket number not set in split")); } - - @Override - public List listPartitionHandles(ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorPartitioningHandle partitioningHandle) - { - HivePartitioningHandle handle = (HivePartitioningHandle) partitioningHandle; - int bucketCount = handle.getBucketCount(); - return IntStream.range(0, bucketCount).mapToObj(HivePartitionHandle::new).collect(toImmutableList()); - } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionHandle.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionHandle.java deleted file mode 100644 index a5d5b1142adf..000000000000 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HivePartitionHandle.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * 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.plugin.hive; - -import io.trino.spi.connector.ConnectorPartitionHandle; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; - -public class HivePartitionHandle - extends ConnectorPartitionHandle -{ - private final int bucket; - - public HivePartitionHandle(int bucket) - { - this.bucket = bucket; - } - - public int getBucket() - { - return bucket; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - HivePartitionHandle that = (HivePartitionHandle) o; - return bucket == that.bucket; - } - - @Override - public int hashCode() - { - return Objects.hash(bucket); - } - - @Override - public String toString() - { - return toStringHelper(this) - .toString(); - } -} diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java index 24e7f31203fe..5e171faa1985 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitManager.java @@ -82,7 +82,6 @@ import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.StandardErrorCode.SERVER_SHUTTING_DOWN; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.GROUPED_SCHEDULING; import static java.lang.Math.min; import static java.lang.String.format; import static java.util.Locale.ENGLISH; @@ -180,7 +179,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { @@ -221,9 +219,6 @@ public ConnectorSplitSource getSplits( // validate bucket bucketed execution Optional bucketHandle = hiveTable.getBucketHandle(); - if ((splitSchedulingStrategy == GROUPED_SCHEDULING) && bucketHandle.isEmpty()) { - throw new TrinoException(GENERIC_INTERNAL_ERROR, "SchedulingPolicy is bucketed, but BucketHandle is not present"); - } // sort partitions partitions = Ordering.natural().onResultOf(HivePartition::getPartitionId).reverse().sortedCopy(partitions); @@ -262,39 +257,18 @@ public ConnectorSplitSource getSplits( .map(validTxnWriteIdList -> validTxnWriteIdList.getTableValidWriteIdList(table.getDatabaseName() + "." + table.getTableName())), hiveTable.getMaxScannedFileSize()); - HiveSplitSource splitSource; - switch (splitSchedulingStrategy) { - case UNGROUPED_SCHEDULING: - splitSource = HiveSplitSource.allAtOnce( - session, - table.getDatabaseName(), - table.getTableName(), - maxInitialSplits, - maxOutstandingSplits, - maxOutstandingSplitsSize, - maxSplitsPerSecond, - hiveSplitLoader, - executor, - highMemorySplitSourceCounter, - hiveTable.isRecordScannedFiles()); - break; - case GROUPED_SCHEDULING: - splitSource = HiveSplitSource.bucketed( - session, - table.getDatabaseName(), - table.getTableName(), - maxInitialSplits, - maxOutstandingSplits, - maxOutstandingSplitsSize, - maxSplitsPerSecond, - hiveSplitLoader, - executor, - highMemorySplitSourceCounter, - hiveTable.isRecordScannedFiles()); - break; - default: - throw new IllegalArgumentException("Unknown splitSchedulingStrategy: " + splitSchedulingStrategy); - } + HiveSplitSource splitSource = HiveSplitSource.allAtOnce( + session, + table.getDatabaseName(), + table.getTableName(), + maxInitialSplits, + maxOutstandingSplits, + maxOutstandingSplitsSize, + maxSplitsPerSecond, + hiveSplitLoader, + executor, + highMemorySplitSourceCounter, + hiveTable.isRecordScannedFiles()); hiveSplitLoader.start(splitSource); return splitSource; diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java index fc627d5b123d..bdd7c970cf89 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/HiveSplitSource.java @@ -25,18 +25,14 @@ import io.trino.plugin.hive.util.SizeBasedSplitWeightProvider; import io.trino.plugin.hive.util.ThrottledAsyncQueue; import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import java.io.FileNotFoundException; import java.util.List; -import java.util.Map; import java.util.Optional; -import java.util.OptionalInt; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -45,7 +41,6 @@ import java.util.function.Function; import java.util.function.Predicate; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.util.concurrent.Futures.immediateVoidFuture; import static io.airlift.concurrent.MoreFutures.failedFuture; @@ -62,7 +57,6 @@ import static io.trino.plugin.hive.HiveSplitSource.StateKind.FAILED; import static io.trino.plugin.hive.HiveSplitSource.StateKind.INITIAL; import static io.trino.plugin.hive.HiveSplitSource.StateKind.NO_MORE_SPLITS; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.lang.String.format; import static java.util.Objects.requireNonNull; @@ -148,16 +142,14 @@ public static HiveSplitSource allAtOnce( private final AsyncQueue queue = new ThrottledAsyncQueue<>(maxSplitsPerSecond, maxOutstandingSplits, executor); @Override - public ListenableFuture offer(OptionalInt bucketNumber, InternalHiveSplit connectorSplit) + public ListenableFuture offer(InternalHiveSplit connectorSplit) { - // bucketNumber can be non-empty because BackgroundHiveSplitLoader does not have knowledge of execution plan return queue.offer(connectorSplit); } @Override - public ListenableFuture borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function, BorrowResult> function) + public ListenableFuture borrowBatchAsync(int maxSize, Function, BorrowResult> function) { - checkArgument(bucketNumber.isEmpty()); return queue.borrowBatchAsync(maxSize, function); } @@ -168,9 +160,8 @@ public void finish() } @Override - public boolean isFinished(OptionalInt bucketNumber) + public boolean isFinished() { - checkArgument(bucketNumber.isEmpty()); return queue.isFinished(); } }, @@ -182,83 +173,6 @@ public boolean isFinished(OptionalInt bucketNumber) recordScannedFiles); } - public static HiveSplitSource bucketed( - ConnectorSession session, - String databaseName, - String tableName, - int estimatedOutstandingSplitsPerBucket, - int maxInitialSplits, - DataSize maxOutstandingSplitsSize, - int maxSplitsPerSecond, - HiveSplitLoader splitLoader, - Executor executor, - CounterStat highMemorySplitSourceCounter, - boolean recordScannedFiles) - { - AtomicReference stateReference = new AtomicReference<>(State.initial()); - return new HiveSplitSource( - session, - databaseName, - tableName, - new PerBucket() - { - private final Map> queues = new ConcurrentHashMap<>(); - private final AtomicBoolean finished = new AtomicBoolean(); - - @Override - public ListenableFuture offer(OptionalInt bucketNumber, InternalHiveSplit connectorSplit) - { - AsyncQueue queue = queueFor(bucketNumber); - queue.offer(connectorSplit); - // Do not block "offer" when running split discovery in bucketed mode. - // A limit is enforced on estimatedSplitSizeInBytes. - return immediateVoidFuture(); - } - - @Override - public ListenableFuture borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function, BorrowResult> function) - { - return queueFor(bucketNumber).borrowBatchAsync(maxSize, function); - } - - @Override - public void finish() - { - if (finished.compareAndSet(false, true)) { - queues.values().forEach(AsyncQueue::finish); - } - } - - @Override - public boolean isFinished(OptionalInt bucketNumber) - { - return queueFor(bucketNumber).isFinished(); - } - - public AsyncQueue queueFor(OptionalInt bucketNumber) - { - checkArgument(bucketNumber.isPresent()); - AtomicBoolean isNew = new AtomicBoolean(); - AsyncQueue queue = queues.computeIfAbsent(bucketNumber.getAsInt(), ignored -> { - isNew.set(true); - return new ThrottledAsyncQueue<>(maxSplitsPerSecond, estimatedOutstandingSplitsPerBucket, executor); - }); - if (isNew.get() && finished.get()) { - // Check `finished` and invoke `queue.finish` after the `queue` is added to the map. - // Otherwise, `queue.finish` may not be invoked if `finished` is set while the lambda above is being evaluated. - queue.finish(); - } - return queue; - } - }, - maxInitialSplits, - maxOutstandingSplitsSize, - splitLoader, - stateReference, - highMemorySplitSourceCounter, - recordScannedFiles); - } - /** * The upper bound of outstanding split count. * It might be larger than the actual number when called concurrently with other methods. @@ -297,8 +211,7 @@ ListenableFuture addToQueue(InternalHiveSplit split) databaseName, tableName, succinctBytes(maxOutstandingSplitsBytes), getBufferedInternalSplitCount())); } bufferedInternalSplitCount.incrementAndGet(); - OptionalInt bucketNumber = split.getReadBucketNumber(); - return queues.offer(bucketNumber, split); + return queues.offer(split); } void noMoreSplits() @@ -327,7 +240,7 @@ void fail(Throwable e) } @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { boolean noMoreSplits; State state = stateReference.get(); @@ -346,8 +259,7 @@ public CompletableFuture getNextBatch(ConnectorPartitionHan throw new UnsupportedOperationException(); } - OptionalInt bucketNumber = toBucketNumber(partitionHandle); - ListenableFuture> future = queues.borrowBatchAsync(bucketNumber, maxSize, internalSplits -> { + ListenableFuture> future = queues.borrowBatchAsync(maxSize, internalSplits -> { ImmutableList.Builder splitsToInsertBuilder = ImmutableList.builder(); ImmutableList.Builder resultBuilder = ImmutableList.builder(); int removedEstimatedSizeInBytes = 0; @@ -448,7 +360,7 @@ else if (maxSplitBytes * 2 >= remainingBlockBytes) { // Side note 2: One could argue that the isEmpty check is overly conservative. // The caller of getNextBatch will likely need to make an extra invocation. // But an extra invocation likely doesn't matter. - return new ConnectorSplitBatch(splits, splits.isEmpty() && queues.isFinished(bucketNumber)); + return new ConnectorSplitBatch(splits, splits.isEmpty() && queues.isFinished()); } else { return new ConnectorSplitBatch(splits, false); @@ -497,14 +409,6 @@ public void close() } } - private static OptionalInt toBucketNumber(ConnectorPartitionHandle partitionHandle) - { - if (partitionHandle == NOT_PARTITIONED) { - return OptionalInt.empty(); - } - return OptionalInt.of(((HivePartitionHandle) partitionHandle).getBucket()); - } - private static boolean setIf(AtomicReference atomicReference, T newValue, Predicate predicate) { while (true) { @@ -531,13 +435,13 @@ private static RuntimeException propagateTrinoException(Throwable throwable) interface PerBucket { - ListenableFuture offer(OptionalInt bucketNumber, InternalHiveSplit split); + ListenableFuture offer(InternalHiveSplit split); - ListenableFuture borrowBatchAsync(OptionalInt bucketNumber, int maxSize, Function, BorrowResult> function); + ListenableFuture borrowBatchAsync(int maxSize, Function, BorrowResult> function); void finish(); - boolean isFinished(OptionalInt bucketNumber); + boolean isFinished(); } static class State diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java index f609800c34e2..41563364f107 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/AbstractTestHive.java @@ -270,9 +270,7 @@ import static io.trino.plugin.hive.util.HiveWriteUtils.getTableDefaultLocation; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.StandardErrorCode.TRANSACTION_CONFLICT; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.MetadataProvider.NOOP_METADATA_PROVIDER; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.connector.RetryMode.NO_RETRIES; import static io.trino.spi.connector.SortOrder.ASC_NULLS_FIRST; import static io.trino.spi.connector.SortOrder.DESC_NULLS_LAST; @@ -5137,7 +5135,7 @@ protected static int getSplitCount(ConnectorSplitSource splitSource) { int splitCount = 0; while (!splitSource.isFinished()) { - splitCount += getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, 1000)).getSplits().size(); + splitCount += getFutureValue(splitSource.getNextBatch(1000)).getSplits().size(); } return splitCount; } @@ -5151,14 +5149,14 @@ protected static List getAllSplits(ConnectorSplitSource splitSou { ImmutableList.Builder splits = ImmutableList.builder(); while (!splitSource.isFinished()) { - splits.addAll(getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, 1000)).getSplits()); + splits.addAll(getFutureValue(splitSource.getNextBatch(1000)).getSplits()); } return splits.build(); } protected static ConnectorSplitSource getSplits(ConnectorSplitManager splitManager, Transaction transaction, ConnectorSession session, ConnectorTableHandle tableHandle) { - return splitManager.getSplits(transaction.getTransactionHandle(), session, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + return splitManager.getSplits(transaction.getTransactionHandle(), session, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue()); } protected String getPartitionId(Object partition) diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java index cb395ad116b1..5152f04e937d 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/BaseHiveConnectorTest.java @@ -98,11 +98,7 @@ import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; import static io.trino.SystemSessionProperties.COLOCATED_JOIN; -import static io.trino.SystemSessionProperties.CONCURRENT_LIFESPANS_PER_NODE; -import static io.trino.SystemSessionProperties.DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION; import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; -import static io.trino.SystemSessionProperties.GROUPED_EXECUTION; -import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static io.trino.SystemSessionProperties.USE_TABLE_SCAN_NODE_PARTITIONING; import static io.trino.plugin.hive.HiveColumnHandle.BUCKET_COLUMN_NAME; import static io.trino.plugin.hive.HiveColumnHandle.FILE_MODIFIED_TIME_COLUMN_NAME; @@ -132,7 +128,6 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.spi.type.VarcharType.createVarcharType; -import static io.trino.sql.planner.OptimizerConfig.JoinDistributionType.BROADCAST; import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static io.trino.sql.planner.planprinter.IoPlanPrinter.FormattedMarker.Bound.ABOVE; import static io.trino.sql.planner.planprinter.IoPlanPrinter.FormattedMarker.Bound.EXACTLY; @@ -5387,7 +5382,7 @@ public void testBucketedSelect() } @Test - public void testGroupedExecution() + public void testGroupedExecution() // TODO change the name { try { assertUpdate( @@ -5405,80 +5400,13 @@ public void testGroupedExecution() "WITH (bucket_count = 13, bucketed_by = ARRAY['key3']) AS\n" + "SELECT orderkey key3, comment value3 FROM orders", 15000); - assertUpdate( - "CREATE TABLE test_grouped_join4\n" + - "WITH (bucket_count = 13, bucketed_by = ARRAY['key4_bucket']) AS\n" + - "SELECT orderkey key4_bucket, orderkey key4_non_bucket, comment value4 FROM orders", - 15000); assertUpdate( "CREATE TABLE test_grouped_joinN AS\n" + "SELECT orderkey keyN, comment valueN FROM orders", 15000); - assertUpdate( - "CREATE TABLE test_grouped_joinDual\n" + - "WITH (bucket_count = 13, bucketed_by = ARRAY['keyD']) AS\n" + - "SELECT orderkey keyD, comment valueD FROM orders CROSS JOIN UNNEST(repeat(NULL, 2))", - 30000); - assertUpdate( - "CREATE TABLE test_grouped_window\n" + - "WITH (bucket_count = 5, bucketed_by = ARRAY['key']) AS\n" + - "SELECT custkey key, orderkey value FROM orders WHERE custkey <= 5 ORDER BY orderkey LIMIT 10", - 10); - // NOT grouped execution; default - Session notColocated = Session.builder(getSession()) + Session session = Session.builder(getSession()) .setSystemProperty(COLOCATED_JOIN, "false") - .setSystemProperty(GROUPED_EXECUTION, "false") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - // Co-located JOIN with all groups at once, fixed schedule - Session colocatedAllGroupsAtOnce = Session.builder(getSession()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "0") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "false") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - // Co-located JOIN, 1 group per worker at a time, fixed schedule - Session colocatedOneGroupAtATime = Session.builder(getSession()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "1") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "false") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - // Co-located JOIN with all groups at once, dynamic schedule - Session colocatedAllGroupsAtOnceDynamic = Session.builder(getSession()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "0") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "true") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - // Co-located JOIN, 1 group per worker at a time, dynamic schedule - Session colocatedOneGroupAtATimeDynamic = Session.builder(getSession()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "1") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "true") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - // Broadcast JOIN, 1 group per worker at a time - Session broadcastOneGroupAtATime = Session.builder(getSession()) - .setSystemProperty(JOIN_DISTRIBUTION_TYPE, BROADCAST.name()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "1") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .build(); - - // Broadcast JOIN, 1 group per worker at a time, dynamic schedule - Session broadcastOneGroupAtATimeDynamic = Session.builder(getSession()) - .setSystemProperty(JOIN_DISTRIBUTION_TYPE, BROADCAST.name()) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "1") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "true") .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") .build(); @@ -5493,13 +5421,6 @@ public void testGroupedExecution() "ON key1 = key2\n" + "JOIN test_grouped_join3\n" + "ON key2 = key3"; - @Language("SQL") String joinThreeMixedTable = - "SELECT key1, value1, key2, value2, keyN, valueN\n" + - "FROM test_grouped_join1\n" + - "JOIN test_grouped_join2\n" + - "ON key1 = key2\n" + - "JOIN test_grouped_joinN\n" + - "ON key2 = keyN"; @Language("SQL") String expectedJoinQuery = "SELECT orderkey, comment, orderkey, comment, orderkey, comment FROM orders"; @Language("SQL") String leftJoinBucketedTable = "SELECT key1, value1, key2, value2\n" + @@ -5513,27 +5434,9 @@ public void testGroupedExecution() "ON key1 = key2"; @Language("SQL") String expectedOuterJoinQuery = "SELECT orderkey, comment, CASE mod(orderkey, 2) WHEN 0 THEN orderkey END, CASE mod(orderkey, 2) WHEN 0 THEN comment END FROM orders"; - assertQuery(notColocated, joinThreeBucketedTable, expectedJoinQuery); - assertQuery(notColocated, leftJoinBucketedTable, expectedOuterJoinQuery); - assertQuery(notColocated, rightJoinBucketedTable, expectedOuterJoinQuery); - - assertQuery(colocatedAllGroupsAtOnce, joinThreeBucketedTable, expectedJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnce, joinThreeMixedTable, expectedJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, joinThreeBucketedTable, expectedJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, joinThreeMixedTable, expectedJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedAllGroupsAtOnceDynamic, joinThreeBucketedTable, expectedJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnceDynamic, joinThreeMixedTable, expectedJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, joinThreeBucketedTable, expectedJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, joinThreeMixedTable, expectedJoinQuery, assertRemoteExchangesCount(2)); - - assertQuery(colocatedAllGroupsAtOnce, leftJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnce, rightJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, leftJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, rightJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnceDynamic, leftJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnceDynamic, rightJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, leftJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, rightJoinBucketedTable, expectedOuterJoinQuery, assertRemoteExchangesCount(1)); + assertQuery(session, joinThreeBucketedTable, expectedJoinQuery); + assertQuery(session, leftJoinBucketedTable, expectedOuterJoinQuery); + assertQuery(session, rightJoinBucketedTable, expectedOuterJoinQuery); // // CROSS JOIN and HASH JOIN mixed @@ -5551,9 +5454,7 @@ public void testGroupedExecution() " (SELECT orderkey key1, comment value1 FROM orders)\n" + "CROSS JOIN\n" + " (SELECT orderkey key3, comment value3 FROM orders WHERE orderkey <= 3)"; - assertQuery(notColocated, crossJoin, expectedCrossJoinQuery); - assertQuery(colocatedAllGroupsAtOnce, crossJoin, expectedCrossJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, crossJoin, expectedCrossJoinQuery, assertRemoteExchangesCount(2)); + assertQuery(session, crossJoin, expectedCrossJoinQuery); // // Bucketed and unbucketed HASH JOIN mixed @@ -5572,178 +5473,7 @@ public void testGroupedExecution() "JOIN test_grouped_join3\n" + "ON key1 = key3"; @Language("SQL") String expectedBucketedAndUnbucketedJoinQuery = "SELECT orderkey, comment, orderkey, comment, orderkey, comment, orderkey, comment FROM orders"; - assertQuery(notColocated, bucketedAndUnbucketedJoin, expectedBucketedAndUnbucketedJoinQuery); - assertQuery(colocatedAllGroupsAtOnce, bucketedAndUnbucketedJoin, expectedBucketedAndUnbucketedJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, bucketedAndUnbucketedJoin, expectedBucketedAndUnbucketedJoinQuery, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, bucketedAndUnbucketedJoin, expectedBucketedAndUnbucketedJoinQuery, assertRemoteExchangesCount(2)); - - // - // UNION ALL / GROUP BY - // ==================== - - @Language("SQL") String groupBySingleBucketed = - "SELECT\n" + - " keyD,\n" + - " count(valueD)\n" + - "FROM\n" + - " test_grouped_joinDual\n" + - "GROUP BY keyD"; - @Language("SQL") String expectedSingleGroupByQuery = "SELECT orderkey, 2 FROM orders"; - @Language("SQL") String groupByOfUnionBucketed = - "SELECT\n" + - " key\n" + - ", arbitrary(value1)\n" + - ", arbitrary(value2)\n" + - ", arbitrary(value3)\n" + - "FROM (\n" + - " SELECT key1 key, value1, NULL value2, NULL value3\n" + - " FROM test_grouped_join1\n" + - "UNION ALL\n" + - " SELECT key2 key, NULL value1, value2, NULL value3\n" + - " FROM test_grouped_join2\n" + - " WHERE key2 % 2 = 0\n" + - "UNION ALL\n" + - " SELECT key3 key, NULL value1, NULL value2, value3\n" + - " FROM test_grouped_join3\n" + - " WHERE key3 % 3 = 0\n" + - ")\n" + - "GROUP BY key"; - @Language("SQL") String groupByOfUnionMixed = - "SELECT\n" + - " key\n" + - ", arbitrary(value1)\n" + - ", arbitrary(value2)\n" + - ", arbitrary(valueN)\n" + - "FROM (\n" + - " SELECT key1 key, value1, NULL value2, NULL valueN\n" + - " FROM test_grouped_join1\n" + - "UNION ALL\n" + - " SELECT key2 key, NULL value1, value2, NULL valueN\n" + - " FROM test_grouped_join2\n" + - " WHERE key2 % 2 = 0\n" + - "UNION ALL\n" + - " SELECT keyN key, NULL value1, NULL value2, valueN\n" + - " FROM test_grouped_joinN\n" + - " WHERE keyN % 3 = 0\n" + - ")\n" + - "GROUP BY key"; - @Language("SQL") String expectedGroupByOfUnion = "SELECT orderkey, comment, CASE mod(orderkey, 2) WHEN 0 THEN comment END, CASE mod(orderkey, 3) WHEN 0 THEN comment END FROM orders"; - // In this case: - // * left side can take advantage of bucketed execution - // * right side does not have the necessary organization to allow its parent to take advantage of bucketed execution - // In this scenario, we give up bucketed execution altogether. This can potentially be improved. - // - // AGG(key) - // | - // UNION ALL - // / \ - // AGG(key) Scan (not bucketed) - // | - // Scan (bucketed on key) - @Language("SQL") String groupByOfUnionOfGroupByMixed = - "SELECT\n" + - " key, sum(cnt) cnt\n" + - "FROM (\n" + - " SELECT keyD key, count(valueD) cnt\n" + - " FROM test_grouped_joinDual\n" + - " GROUP BY keyD\n" + - "UNION ALL\n" + - " SELECT keyN key, 1 cnt\n" + - " FROM test_grouped_joinN\n" + - ")\n" + - "group by key"; - @Language("SQL") String expectedGroupByOfUnionOfGroupBy = "SELECT orderkey, 3 FROM orders"; - - // Eligible GROUP BYs run in the same fragment regardless of colocated_join flag - assertQuery(colocatedAllGroupsAtOnce, groupBySingleBucketed, expectedSingleGroupByQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, groupBySingleBucketed, expectedSingleGroupByQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, groupBySingleBucketed, expectedSingleGroupByQuery, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnce, groupByOfUnionBucketed, expectedGroupByOfUnion, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, groupByOfUnionBucketed, expectedGroupByOfUnion, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, groupByOfUnionBucketed, expectedGroupByOfUnion, assertRemoteExchangesCount(1)); - - // cannot be executed in a grouped manner but should still produce correct result - assertQuery(colocatedOneGroupAtATime, groupByOfUnionMixed, expectedGroupByOfUnion, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, groupByOfUnionOfGroupByMixed, expectedGroupByOfUnionOfGroupBy, assertRemoteExchangesCount(2)); - - // - // GROUP BY and JOIN mixed - // ======================== - @Language("SQL") String joinGroupedWithGrouped = - "SELECT key1, count1, count2\n" + - "FROM (\n" + - " SELECT keyD key1, count(valueD) count1\n" + - " FROM test_grouped_joinDual\n" + - " GROUP BY keyD\n" + - ") JOIN (\n" + - " SELECT keyD key2, count(valueD) count2\n" + - " FROM test_grouped_joinDual\n" + - " GROUP BY keyD\n" + - ")\n" + - "ON key1 = key2"; - @Language("SQL") String expectedJoinGroupedWithGrouped = "SELECT orderkey, 2, 2 FROM orders"; - @Language("SQL") String joinGroupedWithUngrouped = - "SELECT keyD, countD, valueN\n" + - "FROM (\n" + - " SELECT keyD, count(valueD) countD\n" + - " FROM test_grouped_joinDual\n" + - " GROUP BY keyD\n" + - ") JOIN (\n" + - " SELECT keyN, valueN\n" + - " FROM test_grouped_joinN\n" + - ")\n" + - "ON keyD = keyN"; - @Language("SQL") String expectedJoinGroupedWithUngrouped = "SELECT orderkey, 2, comment FROM orders"; - @Language("SQL") String joinUngroupedWithGrouped = - "SELECT keyN, valueN, countD\n" + - "FROM (\n" + - " SELECT keyN, valueN\n" + - " FROM test_grouped_joinN\n" + - ") JOIN (\n" + - " SELECT keyD, count(valueD) countD\n" + - " FROM test_grouped_joinDual\n" + - " GROUP BY keyD\n" + - ")\n" + - "ON keyN = keyD"; - @Language("SQL") String expectedJoinUngroupedWithGrouped = "SELECT orderkey, comment, 2 FROM orders"; - @Language("SQL") String groupOnJoinResult = - "SELECT keyD, count(valueD), count(valueN)\n" + - "FROM\n" + - " test_grouped_joinDual\n" + - "JOIN\n" + - " test_grouped_joinN\n" + - "ON keyD=keyN\n" + - "GROUP BY keyD"; - @Language("SQL") String expectedGroupOnJoinResult = "SELECT orderkey, 2, 2 FROM orders"; - - @Language("SQL") String groupOnUngroupedJoinResult = - "SELECT key4_bucket, count(value4), count(valueN)\n" + - "FROM\n" + - " test_grouped_join4\n" + - "JOIN\n" + - " test_grouped_joinN\n" + - "ON key4_non_bucket=keyN\n" + - "GROUP BY key4_bucket"; - @Language("SQL") String expectedGroupOnUngroupedJoinResult = "SELECT orderkey, count(*), count(*) FROM orders group by orderkey"; - - // Eligible GROUP BYs run in the same fragment regardless of colocated_join flag - assertQuery(colocatedAllGroupsAtOnce, joinGroupedWithGrouped, expectedJoinGroupedWithGrouped, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, joinGroupedWithGrouped, expectedJoinGroupedWithGrouped, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, joinGroupedWithGrouped, expectedJoinGroupedWithGrouped, assertRemoteExchangesCount(1)); - assertQuery(colocatedAllGroupsAtOnce, joinGroupedWithUngrouped, expectedJoinGroupedWithUngrouped, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, joinGroupedWithUngrouped, expectedJoinGroupedWithUngrouped, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, joinGroupedWithUngrouped, expectedJoinGroupedWithUngrouped, assertRemoteExchangesCount(2)); - assertQuery(colocatedAllGroupsAtOnce, groupOnJoinResult, expectedGroupOnJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, groupOnJoinResult, expectedGroupOnJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, groupOnJoinResult, expectedGroupOnJoinResult, assertRemoteExchangesCount(2)); - - assertQuery(broadcastOneGroupAtATime, groupOnJoinResult, expectedGroupOnJoinResult, assertRemoteExchangesCount(2)); - assertQuery(broadcastOneGroupAtATime, groupOnUngroupedJoinResult, expectedGroupOnUngroupedJoinResult, assertRemoteExchangesCount(2)); - assertQuery(broadcastOneGroupAtATimeDynamic, groupOnUngroupedJoinResult, expectedGroupOnUngroupedJoinResult, assertRemoteExchangesCount(2)); - - // cannot be executed in a grouped manner but should still produce correct result - assertQuery(colocatedOneGroupAtATime, joinUngroupedWithGrouped, expectedJoinUngroupedWithGrouped, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, groupOnUngroupedJoinResult, expectedGroupOnUngroupedJoinResult, assertRemoteExchangesCount(4)); + assertQuery(session, bucketedAndUnbucketedJoin, expectedBucketedAndUnbucketedJoinQuery); // // Outer JOIN (that involves LookupOuterOperator) @@ -5769,16 +5499,6 @@ public void testGroupedExecution() " (SELECT * FROM test_grouped_joinN WHERE mod(keyN, 3) = 0)\n" + "ON key1 = keyN"; // The preceding test case, which then feeds into another join - @Language("SQL") String chainedSharedBuildOuterJoin = - "SELECT key1, value1, keyN, valueN, key3, value3\n" + - "FROM\n" + - " (SELECT key1, arbitrary(value1) value1 FROM test_grouped_join1 WHERE mod(key1, 2) = 0 group by key1)\n" + - "RIGHT JOIN\n" + - " (SELECT * FROM test_grouped_joinN WHERE mod(keyN, 3) = 0)\n" + - "ON key1 = keyN\n" + - "FULL JOIN\n" + - " (SELECT * FROM test_grouped_join3 WHERE mod(key3, 5) = 0)\n" + - "ON keyN = key3"; @Language("SQL") String expectedChainedOuterJoinResult = "SELECT\n" + " CASE WHEN mod(orderkey, 2 * 3) = 0 THEN orderkey END,\n" + " CASE WHEN mod(orderkey, 2 * 3) = 0 THEN comment END,\n" + @@ -5796,64 +5516,8 @@ public void testGroupedExecution() "FROM ORDERS\n" + "WHERE mod(orderkey, 3) = 0"; - assertQuery(notColocated, chainedOuterJoin, expectedChainedOuterJoinResult); - assertQuery(colocatedAllGroupsAtOnce, chainedOuterJoin, expectedChainedOuterJoinResult, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, chainedOuterJoin, expectedChainedOuterJoinResult, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATimeDynamic, chainedOuterJoin, expectedChainedOuterJoinResult, assertRemoteExchangesCount(1)); - assertQuery(notColocated, sharedBuildOuterJoin, expectedSharedBuildOuterJoinResult); - assertQuery(colocatedAllGroupsAtOnce, sharedBuildOuterJoin, expectedSharedBuildOuterJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, sharedBuildOuterJoin, expectedSharedBuildOuterJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, sharedBuildOuterJoin, expectedSharedBuildOuterJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATime, chainedSharedBuildOuterJoin, expectedChainedOuterJoinResult, assertRemoteExchangesCount(2)); - assertQuery(colocatedOneGroupAtATimeDynamic, chainedSharedBuildOuterJoin, expectedChainedOuterJoinResult, assertRemoteExchangesCount(2)); - - // - // Window function - // =============== - assertQuery( - colocatedOneGroupAtATime, - "SELECT key, count(*) OVER (PARTITION BY key ORDER BY value) FROM test_grouped_window", - "VALUES\n" + - "(1, 1),\n" + - "(2, 1),\n" + - "(2, 2),\n" + - "(4, 1),\n" + - "(4, 2),\n" + - "(4, 3),\n" + - "(4, 4),\n" + - "(4, 5),\n" + - "(5, 1),\n" + - "(5, 2)", - assertRemoteExchangesCount(1)); - - assertQuery( - colocatedOneGroupAtATime, - "SELECT key, row_number() OVER (PARTITION BY key ORDER BY value) FROM test_grouped_window", - "VALUES\n" + - "(1, 1),\n" + - "(2, 1),\n" + - "(2, 2),\n" + - "(4, 1),\n" + - "(4, 2),\n" + - "(4, 3),\n" + - "(4, 4),\n" + - "(4, 5),\n" + - "(5, 1),\n" + - "(5, 2)", - assertRemoteExchangesCount(1)); - - assertQuery( - colocatedOneGroupAtATime, - "SELECT key, n FROM (SELECT key, row_number() OVER (PARTITION BY key ORDER BY value) AS n FROM test_grouped_window) WHERE n <= 2", - "VALUES\n" + - "(1, 1),\n" + - "(2, 1),\n" + - "(2, 2),\n" + - "(4, 1),\n" + - "(4, 2),\n" + - "(5, 1),\n" + - "(5, 2)", - assertRemoteExchangesCount(1)); + assertQuery(session, chainedOuterJoin, expectedChainedOuterJoinResult); + assertQuery(session, sharedBuildOuterJoin, expectedSharedBuildOuterJoinResult); // // Filter out all or majority of splits @@ -5885,21 +5549,14 @@ public void testGroupedExecution() "FROM ORDERS\n" + "WHERE mod(orderkey, 13) IN (1, 11)"; - assertQuery(notColocated, noSplits, expectedNoSplits); - assertQuery(colocatedAllGroupsAtOnce, noSplits, expectedNoSplits, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, noSplits, expectedNoSplits, assertRemoteExchangesCount(1)); - assertQuery(notColocated, joinMismatchedBuckets, expectedJoinMismatchedBuckets); - assertQuery(colocatedAllGroupsAtOnce, joinMismatchedBuckets, expectedJoinMismatchedBuckets, assertRemoteExchangesCount(1)); - assertQuery(colocatedOneGroupAtATime, joinMismatchedBuckets, expectedJoinMismatchedBuckets, assertRemoteExchangesCount(1)); + assertQuery(session, noSplits, expectedNoSplits); + assertQuery(session, joinMismatchedBuckets, expectedJoinMismatchedBuckets); } finally { assertUpdate("DROP TABLE IF EXISTS test_grouped_join1"); assertUpdate("DROP TABLE IF EXISTS test_grouped_join2"); assertUpdate("DROP TABLE IF EXISTS test_grouped_join3"); - assertUpdate("DROP TABLE IF EXISTS test_grouped_join4"); assertUpdate("DROP TABLE IF EXISTS test_grouped_joinN"); - assertUpdate("DROP TABLE IF EXISTS test_grouped_joinDual"); - assertUpdate("DROP TABLE IF EXISTS test_grouped_window"); } } diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java index 18e5c163e45a..71647dda3ced 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestBackgroundHiveSplitLoader.java @@ -118,7 +118,6 @@ import static io.trino.plugin.hive.acid.AcidTransaction.NO_ACID_TRANSACTION; import static io.trino.plugin.hive.util.HiveBucketing.BucketingVersion.BUCKETING_V1; import static io.trino.plugin.hive.util.HiveUtil.getRegularColumnHandles; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.predicate.TupleDomain.withColumnDomains; import static io.trino.spi.type.IntegerType.INTEGER; import static io.trino.spi.type.VarcharType.VARCHAR; @@ -960,7 +959,7 @@ private static List drainSplits(HiveSplitSource source) while (!source.isFinished()) { ConnectorSplitBatch batch; try { - batch = source.getNextBatch(NOT_PARTITIONED, 100).get(); + batch = source.getNextBatch(100).get(); } catch (ExecutionException e) { throwIfUnchecked(e.getCause()); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java index 1f1a393fbbd3..5b97b55c177e 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestHiveSplitSource.java @@ -36,7 +36,6 @@ import static io.trino.plugin.hive.HiveErrorCode.HIVE_EXCEEDED_SPLIT_BUFFERING_LIMIT; import static io.trino.plugin.hive.HiveSessionProperties.getMaxInitialSplitSize; import static io.trino.plugin.hive.HiveTestUtils.SESSION; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.testing.assertions.TrinoExceptionAssert.assertTrinoExceptionThrownBy; import static java.lang.Math.toIntExact; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -303,42 +302,9 @@ public void testOutstandingSplitSize() .hasMessageContaining("Split buffering for database.table exceeded memory limit"); } - @Test - public void testEmptyBucket() - { - HiveSplitSource hiveSplitSource = HiveSplitSource.bucketed( - SESSION, - "database", - "table", - 10, - 10, - DataSize.of(1, MEGABYTE), - Integer.MAX_VALUE, - new TestingHiveSplitLoader(), - Executors.newFixedThreadPool(5), - new CounterStat(), - false); - hiveSplitSource.addToQueue(new TestSplit(0, OptionalInt.of(2))); - hiveSplitSource.noMoreSplits(); - assertEquals(getSplits(hiveSplitSource, OptionalInt.of(0), 10).size(), 0); - assertEquals(getSplits(hiveSplitSource, OptionalInt.of(1), 10).size(), 0); - assertEquals(getSplits(hiveSplitSource, OptionalInt.of(2), 10).size(), 1); - assertEquals(getSplits(hiveSplitSource, OptionalInt.of(3), 10).size(), 0); - } - private static List getSplits(ConnectorSplitSource source, int maxSize) { - return getSplits(source, OptionalInt.empty(), maxSize); - } - - private static List getSplits(ConnectorSplitSource source, OptionalInt bucketNumber, int maxSize) - { - if (bucketNumber.isPresent()) { - return getFutureValue(source.getNextBatch(new HivePartitionHandle(bucketNumber.getAsInt()), maxSize)).getSplits(); - } - else { - return getFutureValue(source.getNextBatch(NOT_PARTITIONED, maxSize)).getSplits(); - } + return getFutureValue(source.getNextBatch(maxSize)).getSplits(); } private static class TestingHiveSplitLoader diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java index f3597d29e1f6..501a0f801813 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestOrcPageSourceMemoryTracking.java @@ -19,7 +19,6 @@ import io.airlift.stats.Distribution; import io.airlift.units.DataSize; import io.trino.connector.CatalogName; -import io.trino.execution.Lifespan; import io.trino.hive.orc.NullMemoryManager; import io.trino.hive.orc.impl.WriterImpl; import io.trino.metadata.FunctionManager; @@ -599,7 +598,7 @@ public SourceOperator newTableScanOperator(DriverContext driverContext) columns.stream().map(ColumnHandle.class::cast).collect(toImmutableList()), DynamicFilter.EMPTY); SourceOperator operator = sourceOperatorFactory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); return operator; } @@ -626,7 +625,7 @@ public SourceOperator newScanFilterAndProjectOperator(DriverContext driverContex DataSize.ofBytes(0), 0); SourceOperator operator = sourceOperatorFactory.createOperator(driverContext); - operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit(), Lifespan.taskWide())); + operator.addSplit(new Split(new CatalogName("test"), TestingSplit.createLocalSplit())); operator.noMoreSplits(); return operator; } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java index d30dcb56f694..831b88e4848d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java @@ -54,7 +54,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle handle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 38115a8b0aad..a046c90439ee 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -25,7 +25,6 @@ import io.trino.spi.SplitWeight; import io.trino.spi.TrinoException; import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitSource; import io.trino.spi.connector.Constraint; @@ -135,7 +134,7 @@ public IcebergSplitSource( } @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { long timeLeft = dynamicFilteringWaitTimeoutMillis - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); if (dynamicFilter.isAwaitable() && timeLeft > 0) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 0c4a383148ce..dbbfef8ed2a9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -191,7 +191,7 @@ public TupleDomain getCurrentPredicate() new IcebergConfig().getMinimumAssignedSplitWeight())) { ImmutableList.Builder splits = ImmutableList.builder(); while (!splitSource.isFinished()) { - splitSource.getNextBatch(null, 100).get() + splitSource.getNextBatch(100).get() .getSplits() .stream() .map(IcebergSplit.class::cast) diff --git a/plugin/trino-jmx/src/main/java/io/trino/plugin/jmx/JmxSplitManager.java b/plugin/trino-jmx/src/main/java/io/trino/plugin/jmx/JmxSplitManager.java index 22a69e7325c8..61b93deb96ea 100644 --- a/plugin/trino-jmx/src/main/java/io/trino/plugin/jmx/JmxSplitManager.java +++ b/plugin/trino-jmx/src/main/java/io/trino/plugin/jmx/JmxSplitManager.java @@ -58,7 +58,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java index ac112765c379..369de4b8b404 100644 --- a/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java +++ b/plugin/trino-jmx/src/test/java/io/trino/plugin/jmx/TestJmxSplitManager.java @@ -49,8 +49,6 @@ import static io.airlift.slice.Slices.utf8Slice; import static io.trino.plugin.jmx.JmxMetadata.HISTORY_SCHEMA_NAME; import static io.trino.plugin.jmx.JmxMetadata.JMX_SCHEMA_NAME; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.TimestampWithTimeZoneType.createTimestampWithTimeZoneType; import static io.trino.spi.type.VarcharType.createUnboundedVarcharType; import static io.trino.testing.TestingConnectorSession.SESSION; @@ -106,7 +104,7 @@ public void testPredicatePushdown() TupleDomain nodeTupleDomain = TupleDomain.fromFixedValues(ImmutableMap.of(columnHandle, NullableValue.of(createUnboundedVarcharType(), utf8Slice(nodeIdentifier)))); JmxTableHandle tableHandle = new JmxTableHandle(new SchemaTableName("schema", "tableName"), ImmutableList.of("objectName"), ImmutableList.of(columnHandle), true, nodeTupleDomain); - ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue()); List allSplits = getAllSplits(splitSource); assertEquals(allSplits.size(), 1); @@ -120,7 +118,7 @@ public void testNoPredicate() throws Exception { JmxTableHandle tableHandle = new JmxTableHandle(new SchemaTableName("schema", "tableName"), ImmutableList.of("objectName"), ImmutableList.of(columnHandle), true, TupleDomain.all()); - ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue()); List allSplits = getAllSplits(splitSource); assertEquals(allSplits.size(), nodes.size()); @@ -196,7 +194,7 @@ private RecordSet getRecordSet(SchemaTableName schemaTableName) JmxTableHandle tableHandle = metadata.getTableHandle(SESSION, schemaTableName); List columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(SESSION, tableHandle).values()); - ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + ConnectorSplitSource splitSource = splitManager.getSplits(JmxTransactionHandle.INSTANCE, SESSION, tableHandle, DynamicFilter.EMPTY, Constraint.alwaysTrue()); List allSplits = getAllSplits(splitSource); assertEquals(allSplits.size(), nodes.size()); ConnectorSplit split = allSplits.get(0); @@ -209,7 +207,7 @@ private static List getAllSplits(ConnectorSplitSource splitSourc { ImmutableList.Builder splits = ImmutableList.builder(); while (!splitSource.isFinished()) { - splits.addAll(splitSource.getNextBatch(NOT_PARTITIONED, 1000).get().getSplits()); + splits.addAll(splitSource.getNextBatch(1000).get().getSplits()); } return splits.build(); } diff --git a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSplitManager.java b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSplitManager.java index cbdd453a0644..081690e7d30f 100644 --- a/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSplitManager.java +++ b/plugin/trino-kafka/src/main/java/io/trino/plugin/kafka/KafkaSplitManager.java @@ -62,7 +62,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-kinesis/src/main/java/io/trino/plugin/kinesis/KinesisSplitManager.java b/plugin/trino-kinesis/src/main/java/io/trino/plugin/kinesis/KinesisSplitManager.java index 516e76ffce6e..9ffc74655c19 100644 --- a/plugin/trino-kinesis/src/main/java/io/trino/plugin/kinesis/KinesisSplitManager.java +++ b/plugin/trino-kinesis/src/main/java/io/trino/plugin/kinesis/KinesisSplitManager.java @@ -100,7 +100,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduBucketedSplitSource.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduBucketedSplitSource.java deleted file mode 100644 index 5db9f1d4f200..000000000000 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduBucketedSplitSource.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.plugin.kudu; - -import com.google.common.collect.ImmutableList; -import io.trino.spi.connector.ConnectorPartitionHandle; -import io.trino.spi.connector.ConnectorSplitSource; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; - -import static java.util.concurrent.CompletableFuture.completedFuture; -import static java.util.stream.Collectors.groupingBy; - -public class KuduBucketedSplitSource - implements ConnectorSplitSource -{ - private final Map> groupedSplits; - - KuduBucketedSplitSource(List splits) - { - Map> map = splits.stream() - .collect(groupingBy(KuduSplit::getBucketNumber)); - this.groupedSplits = new ConcurrentHashMap<>(map); - } - - @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) - { - KuduPartitionHandle kuduPartitionHandle = (KuduPartitionHandle) partitionHandle; - List kuduSplits = groupedSplits.remove(kuduPartitionHandle.getBucket()); - return completedFuture(new ConnectorSplitBatch(kuduSplits == null ? ImmutableList.of() : ImmutableList.copyOf(kuduSplits), true)); - } - - @Override - public void close() - { - } - - @Override - public boolean isFinished() - { - return groupedSplits.isEmpty(); - } -} diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduClientConfig.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduClientConfig.java index 6e2e70158003..1f8254cf77c4 100755 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduClientConfig.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduClientConfig.java @@ -44,7 +44,6 @@ public class KuduClientConfig private boolean disableStatistics; private boolean schemaEmulationEnabled; private String schemaEmulationPrefix = "presto::"; - private boolean groupedExecutionEnabled; private Duration dynamicFilteringWaitTimeout = new Duration(0, MINUTES); @NotNull @@ -131,18 +130,6 @@ public KuduClientConfig setSchemaEmulationEnabled(boolean enabled) return this; } - @Config("kudu.grouped-execution.enabled") - public KuduClientConfig setGroupedExecutionEnabled(boolean enabled) - { - this.groupedExecutionEnabled = enabled; - return this; - } - - public boolean isGroupedExecutionEnabled() - { - return groupedExecutionEnabled; - } - @MinDuration("0ms") @NotNull public Duration getDynamicFilteringWaitTimeout() diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduMetadata.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduMetadata.java index 306107748f7b..45b2be25c817 100755 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduMetadata.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduMetadata.java @@ -15,11 +15,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import io.airlift.slice.Slice; import io.trino.plugin.kudu.properties.KuduTableProperties; import io.trino.plugin.kudu.properties.PartitionDesign; -import io.trino.plugin.kudu.schema.KuduRangePartition; import io.trino.spi.TrinoException; import io.trino.spi.connector.Assignment; import io.trino.spi.connector.ColumnHandle; @@ -53,7 +51,6 @@ import org.apache.kudu.ColumnSchema; import org.apache.kudu.Schema; import org.apache.kudu.client.KuduTable; -import org.apache.kudu.client.Partition; import org.apache.kudu.client.PartitionSchema.HashBucketSchema; import javax.inject.Inject; @@ -68,12 +65,9 @@ import java.util.OptionalInt; import java.util.OptionalLong; import java.util.Set; -import java.util.stream.Collectors; import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.plugin.kudu.KuduColumnHandle.ROW_ID; -import static io.trino.plugin.kudu.KuduSessionProperties.isKuduGroupedExecutionEnabled; -import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.connector.RetryMode.NO_RETRIES; import static java.util.Objects.requireNonNull; @@ -392,11 +386,6 @@ public Optional finishCreateTable( return Optional.empty(); } - private static boolean isTableSupportGroupedExecution(KuduTable kuduTable) - { - return !kuduTable.getPartitionSchema().getHashBucketSchemas().isEmpty(); - } - @Override public ColumnHandle getDeleteRowIdColumnHandle(ConnectorSession session, ConnectorTableHandle tableHandle) { @@ -428,33 +417,11 @@ public void finishDelete(ConnectorSession session, ConnectorTableHandle tableHan public ConnectorTableProperties getTableProperties(ConnectorSession session, ConnectorTableHandle table) { KuduTableHandle handle = (KuduTableHandle) table; - KuduTable kuduTable = handle.getTable(clientSession); Optional tablePartitioning = Optional.empty(); Optional> partitioningColumns = Optional.empty(); List> localProperties = ImmutableList.of(); - if (isKuduGroupedExecutionEnabled(session) - && isTableSupportGroupedExecution(kuduTable)) { - Map columnMap = getColumnHandles(session, handle); - List bucketColumnIds = getBucketColumnIds(kuduTable); - List bucketColumns = getSpecifyColumns(kuduTable.getSchema(), bucketColumnIds, columnMap); - Optional> kuduRangePartitions = getKuduRangePartitions(kuduTable); - tablePartitioning = Optional.of(new ConnectorTablePartitioning( - new KuduPartitioningHandle( - handle.getSchemaTableName().getSchemaName(), - handle.getSchemaTableName().getTableName(), - handle.getBucketCount().orElse(0), - bucketColumnIds, - bucketColumns.stream() - .map(KuduColumnHandle.class::cast) - .map(KuduColumnHandle::getType) - .collect(Collectors.toList()), - kuduRangePartitions), - bucketColumns)); - partitioningColumns = Optional.of(ImmutableSet.copyOf(bucketColumns)); - } - return new ConnectorTableProperties( handle.getConstraint(), tablePartitioning, @@ -463,24 +430,6 @@ && isTableSupportGroupedExecution(kuduTable)) { localProperties); } - private List getSpecifyColumns(Schema schema, List targetColumns, Map columnMap) - { - return targetColumns.stream() - .map(schema::getColumnByIndex) - .map(ColumnSchema::getName) - .map(columnMap::get) - .collect(toImmutableList()); - } - - private List getBucketColumnIds(KuduTable kuduTable) - { - List hashBucketSchemas = kuduTable.getPartitionSchema().getHashBucketSchemas(); - return hashBucketSchemas.stream() - .map(HashBucketSchema::getColumnIds) - .flatMap(List::stream) - .collect(toImmutableList()); - } - @Override public Optional> applyFilter(ConnectorSession session, ConnectorTableHandle table, Constraint constraint) { @@ -582,24 +531,4 @@ public Optional> applyLimit(Connect return Optional.of(new LimitApplicationResult<>(handle, false, false)); } - - private static Optional> getKuduRangePartitions(KuduTable table) - { - List rangePartitions = getRangePartitions(table); - List kuduRangePartitions = rangePartitions.stream() - .map(partition -> new KuduRangePartition(partition.getRangeKeyStart(), partition.getRangeKeyEnd())) - .collect(toImmutableList()); - return kuduRangePartitions.isEmpty() ? Optional.empty() : Optional.of(kuduRangePartitions); - } - - private static List getRangePartitions(KuduTable table) - { - final long fetchTabletsTimeoutInMillis = 60 * 1000; - try { - return table.getRangePartitions(fetchTabletsTimeoutInMillis); - } - catch (Exception e) { - throw new TrinoException(GENERIC_INTERNAL_ERROR, "Unable to get list of tablets for table " + table.getName(), e); - } - } } diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduNodePartitioningProvider.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduNodePartitioningProvider.java index 79c7c2ca12ed..d12b809192d3 100644 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduNodePartitioningProvider.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduNodePartitioningProvider.java @@ -16,7 +16,6 @@ import io.trino.spi.connector.BucketFunction; import io.trino.spi.connector.ConnectorBucketNodeMap; import io.trino.spi.connector.ConnectorNodePartitioningProvider; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorPartitioningHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; @@ -28,9 +27,7 @@ import java.util.List; import java.util.function.ToIntFunction; -import java.util.stream.IntStream; -import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.spi.connector.ConnectorBucketNodeMap.createBucketNodeMap; import static java.util.Objects.requireNonNull; @@ -45,18 +42,6 @@ public KuduNodePartitioningProvider(KuduClientSession clientSession) this.clientSession = requireNonNull(clientSession, "clientSession is null"); } - @Override - public List listPartitionHandles( - ConnectorTransactionHandle transactionHandle, - ConnectorSession session, - ConnectorPartitioningHandle partitioningHandle) - { - KuduPartitioningHandle handle = (KuduPartitioningHandle) partitioningHandle; - return IntStream.range(0, handle.getBucketCount()) - .mapToObj(KuduPartitionHandle::new) - .collect(toImmutableList()); - } - @Override public ConnectorBucketNodeMap getBucketNodeMap( ConnectorTransactionHandle transactionHandle, diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduPartitionHandle.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduPartitionHandle.java deleted file mode 100644 index 8f9ec545e346..000000000000 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduPartitionHandle.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.plugin.kudu; - -import io.trino.spi.connector.ConnectorPartitionHandle; - -import java.util.Objects; - -import static com.google.common.base.MoreObjects.toStringHelper; - -public class KuduPartitionHandle - extends ConnectorPartitionHandle -{ - private final int bucket; - - public KuduPartitionHandle(int bucket) - { - this.bucket = bucket; - } - - public int getBucket() - { - return bucket; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - KuduPartitionHandle that = (KuduPartitionHandle) o; - return bucket == that.bucket; - } - - @Override - public int hashCode() - { - return Objects.hash(bucket); - } - - @Override - public String toString() - { - return toStringHelper(this) - .add("bucket", bucket) - .toString(); - } -} diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSessionProperties.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSessionProperties.java index 98e9df279ac9..9c600ab01330 100644 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSessionProperties.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSessionProperties.java @@ -23,11 +23,9 @@ import java.util.List; import static io.trino.plugin.base.session.PropertyMetadataUtil.durationProperty; -import static io.trino.spi.session.PropertyMetadata.booleanProperty; public final class KuduSessionProperties { - private static final String KUDU_GROUPED_EXECUTION_ENABLED = "grouped_execution"; private static final String DYNAMIC_FILTERING_WAIT_TIMEOUT = "dynamic_filtering_wait_timeout"; private final List> sessionProperties; @@ -36,11 +34,6 @@ public final class KuduSessionProperties public KuduSessionProperties(KuduClientConfig kuduConfig) { sessionProperties = ImmutableList.of( - booleanProperty( - KUDU_GROUPED_EXECUTION_ENABLED, - "Enable grouped execution.", - kuduConfig.isGroupedExecutionEnabled(), - false), durationProperty( DYNAMIC_FILTERING_WAIT_TIMEOUT, "Duration to wait for completion of dynamic filters", @@ -53,11 +46,6 @@ public List> getSessionProperties() return sessionProperties; } - public static boolean isKuduGroupedExecutionEnabled(ConnectorSession session) - { - return session.getProperty(KUDU_GROUPED_EXECUTION_ENABLED, Boolean.class); - } - public static Duration getDynamicFilteringWaitTimeout(ConnectorSession session) { return session.getProperty(DYNAMIC_FILTERING_WAIT_TIMEOUT, Duration.class); diff --git a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSplitManager.java b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSplitManager.java index 059542ac1cd7..0552eadff4e4 100755 --- a/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSplitManager.java +++ b/plugin/trino-kudu/src/main/java/io/trino/plugin/kudu/KuduSplitManager.java @@ -13,7 +13,6 @@ */ package io.trino.plugin.kudu; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplitManager; import io.trino.spi.connector.ConnectorSplitSource; @@ -50,37 +49,29 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { long timeoutMillis = getDynamicFilteringWaitTimeout(session).toMillis(); if (timeoutMillis == 0 || !dynamicFilter.isAwaitable()) { - return getSplitSource(table, splitSchedulingStrategy, dynamicFilter); + return getSplitSource(table, dynamicFilter); } CompletableFuture dynamicFilterFuture = whenCompleted(dynamicFilter) .completeOnTimeout(null, timeoutMillis, MILLISECONDS); CompletableFuture splitSourceFuture = dynamicFilterFuture.thenApply( - ignored -> getSplitSource(table, splitSchedulingStrategy, dynamicFilter)); + ignored -> getSplitSource(table, dynamicFilter)); return new KuduDynamicFilteringSplitSource(dynamicFilterFuture, splitSourceFuture); } private ConnectorSplitSource getSplitSource( ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter) { KuduTableHandle handle = (KuduTableHandle) table; List splits = clientSession.buildKuduSplits(handle, dynamicFilter); - switch (splitSchedulingStrategy) { - case UNGROUPED_SCHEDULING: - return new FixedSplitSource(splits); - case GROUPED_SCHEDULING: - return new KuduBucketedSplitSource(splits); - } - throw new IllegalArgumentException("Unknown splitSchedulingStrategy: " + splitSchedulingStrategy); + return new FixedSplitSource(splits); } private static CompletableFuture whenCompleted(DynamicFilter dynamicFilter) @@ -106,9 +97,9 @@ private KuduDynamicFilteringSplitSource( } @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { - return splitSourceFuture.thenCompose(splitSource -> splitSource.getNextBatch(partitionHandle, maxSize)); + return splitSourceFuture.thenCompose(splitSource -> splitSource.getNextBatch(maxSize)); } @Override diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduClientConfig.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduClientConfig.java index d0985923ae4c..4e57ba7df21c 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduClientConfig.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduClientConfig.java @@ -37,7 +37,6 @@ public void testDefaults() .setDisableStatistics(false) .setSchemaEmulationEnabled(false) .setSchemaEmulationPrefix("presto::") - .setGroupedExecutionEnabled(false) .setDynamicFilteringWaitTimeout(new Duration(0, MINUTES))); } @@ -51,7 +50,6 @@ public void testExplicitPropertyMappingsWithCredentialsKey() .put("kudu.client.disable-statistics", "true") .put("kudu.schema-emulation.enabled", "true") .put("kudu.schema-emulation.prefix", "trino::") - .put("kudu.grouped-execution.enabled", "true") .put("kudu.dynamic-filtering.wait-timeout", "30m") .buildOrThrow(); @@ -62,7 +60,6 @@ public void testExplicitPropertyMappingsWithCredentialsKey() .setDisableStatistics(true) .setSchemaEmulationEnabled(true) .setSchemaEmulationPrefix("trino::") - .setGroupedExecutionEnabled(true) .setDynamicFilteringWaitTimeout(new Duration(30, MINUTES)); assertFullMapping(properties, expected); diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDynamicFilter.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDynamicFilter.java index 5a1305119ee5..386c995bcde6 100644 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDynamicFilter.java +++ b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationDynamicFilter.java @@ -17,7 +17,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.primitives.Ints; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.execution.QueryStats; import io.trino.metadata.QualifiedObjectName; import io.trino.metadata.Split; @@ -52,9 +51,7 @@ import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; import static io.trino.SystemSessionProperties.JOIN_REORDERING_STRATEGY; import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createKuduQueryRunnerTpch; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.sql.planner.OptimizerConfig.JoinDistributionType.BROADCAST; import static io.trino.sql.planner.OptimizerConfig.JoinReorderingStrategy.NONE; import static org.testng.Assert.assertEquals; @@ -105,10 +102,10 @@ public void testIncompleteDynamicFilterTimeout() Optional tableHandle = runner.getMetadata().getTableHandle(session, tableName); assertTrue(tableHandle.isPresent()); SplitSource splitSource = runner.getSplitManager() - .getSplits(session, tableHandle.get(), UNGROUPED_SCHEDULING, new IncompleteDynamicFilter(), alwaysTrue()); + .getSplits(session, tableHandle.get(), new IncompleteDynamicFilter(), alwaysTrue()); List splits = new ArrayList<>(); while (!splitSource.isFinished()) { - splits.addAll(splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), 1000).get().getSplits()); + splits.addAll(splitSource.getNextBatch(1000).get().getSplits()); } splitSource.close(); assertFalse(splits.isEmpty()); diff --git a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationGroupedExecution.java b/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationGroupedExecution.java deleted file mode 100644 index 8c8b5e3960d9..000000000000 --- a/plugin/trino-kudu/src/test/java/io/trino/plugin/kudu/TestKuduIntegrationGroupedExecution.java +++ /dev/null @@ -1,241 +0,0 @@ -/* - * 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.plugin.kudu; - -import io.trino.Session; -import io.trino.cost.StatsAndCosts; -import io.trino.metadata.FunctionManager; -import io.trino.metadata.Metadata; -import io.trino.sql.planner.Plan; -import io.trino.sql.planner.plan.ExchangeNode; -import io.trino.testing.AbstractTestQueryFramework; -import io.trino.testing.QueryRunner; -import org.testng.annotations.AfterClass; -import org.testng.annotations.Test; - -import java.util.function.Consumer; - -import static io.trino.SystemSessionProperties.COLOCATED_JOIN; -import static io.trino.SystemSessionProperties.CONCURRENT_LIFESPANS_PER_NODE; -import static io.trino.SystemSessionProperties.DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION; -import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; -import static io.trino.SystemSessionProperties.GROUPED_EXECUTION; -import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createKuduQueryRunner; -import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createSession; -import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; -import static io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan; -import static io.trino.testing.sql.TestTable.randomTableSuffix; -import static java.lang.String.format; - -public class TestKuduIntegrationGroupedExecution - extends AbstractTestQueryFramework -{ - private static final String SCHEMA_KUDU = "kudu"; - private static final String KUDU_GROUPED_EXECUTION = "grouped_execution"; - private TestingKuduServer kuduServer; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - kuduServer = new TestingKuduServer(); - Session groupedExecutionSessionProperties = Session.builder(createSession("test_grouped_execution")) - .setSystemProperty(COLOCATED_JOIN, "true") - .setSystemProperty(GROUPED_EXECUTION, "true") - .setSystemProperty(CONCURRENT_LIFESPANS_PER_NODE, "1") - .setSystemProperty(DYNAMIC_SCHEDULE_FOR_GROUPED_EXECUTION, "false") - .setSystemProperty(ENABLE_DYNAMIC_FILTERING, "false") - .setCatalogSessionProperty(SCHEMA_KUDU, KUDU_GROUPED_EXECUTION, "true") - .build(); - return createKuduQueryRunner(kuduServer, groupedExecutionSessionProperties); - } - - @AfterClass(alwaysRun = true) - public final void destroy() - { - if (kuduServer != null) { - kuduServer.close(); - kuduServer = null; - } - } - - @Test - public void testGroupedExecutionJoin() - { - assertUpdate("CREATE TABLE IF NOT EXISTS test_grouped_execution_t1 (" + - "key1 INT WITH (primary_key=true), " + - "key2 INT WITH (primary_key=true), " + - "attr1 INT" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1'], " + - " partition_by_hash_buckets = 2" + - ")"); - - assertUpdate("CREATE TABLE IF NOT EXISTS test_grouped_execution_t2 (" + - "key1 INT WITH (primary_key=true), " + - "key2 INT WITH (primary_key=true), " + - "attr2 decimal(10, 6)" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1'], " + - " partition_by_hash_buckets = 2" + - ")"); - - assertUpdate("INSERT INTO test_grouped_execution_t1 VALUES (0, 0, 0), (0, 1, 0), (1, 1, 1)", 3); - assertUpdate("INSERT INTO test_grouped_execution_t2 VALUES (0, 0, 0), (1, 1, 1), (1, 2, 1)", 3); - assertQuery( - getSession(), - "SELECT t1.* FROM test_grouped_execution_t1 t1 join test_grouped_execution_t2 t2 on t1.key1=t2.key1 WHERE t1.attr1=0", - "VALUES (0, 0, 0), (0, 1, 0)", - assertRemoteExchangesCount(1)); - - assertUpdate("DROP TABLE test_grouped_execution_t1"); - assertUpdate("DROP TABLE test_grouped_execution_t2"); - } - - @Test - public void testGroupedExecutionJoinRangePartition() - { - String tableName1 = "test_grouped_execution_range_t1_" + randomTableSuffix(); - assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName1 + " (" + - "key1 INT WITH (primary_key=true), " + - "key2 INT WITH (primary_key=true), " + - "attr1 INT" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1'], " + - " partition_by_hash_buckets = 2, " + - " partition_by_range_columns = ARRAY['key2']," + - " range_partitions = '[{\"lower\": null, \"upper\": \"4\"}, {\"lower\": \"4\", \"upper\": null}]'" + - ")"); - - String tableName2 = "test_grouped_execution_range_t2_" + randomTableSuffix(); - assertUpdate("CREATE TABLE IF NOT EXISTS " + tableName2 + " (" + - "key1 INT WITH (primary_key=true), " + - "key2 INT WITH (primary_key=true), " + - "attr2 decimal(10, 6)" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1'], " + - " partition_by_hash_buckets = 2," + - " partition_by_range_columns = ARRAY['key2']," + - " range_partitions = '[{\"lower\": null, \"upper\": \"4\"}, {\"lower\": \"4\", \"upper\": null}]'" + - ")"); - - assertUpdate("INSERT INTO " + tableName1 + " VALUES (0, 0, 0), (0, 5, 0), (1, 0, 0), (1, 5, 0)", 4); - assertUpdate("INSERT INTO " + tableName2 + " VALUES (0, 0, 0), (0, 5, 1), (1, 0, 0), (1, 5, 2)", 4); - assertQuery( - getSession(), - "SELECT t1.* FROM " + tableName1 + " t1 join " + tableName2 + " t2 on t1.key1=t2.key1 WHERE t2.attr2=2", - "VALUES (1, 0, 0), (1, 5, 0)", - assertRemoteExchangesCount(1)); - - assertUpdate("DROP TABLE " + tableName1); - assertUpdate("DROP TABLE " + tableName2); - } - - @Test - public void testGroupedExecutionGroupBy() - { - assertUpdate("CREATE TABLE IF NOT EXISTS test_grouped_execution (" + - "key1 INT WITH (primary_key=true), " + - "key2 INT WITH (primary_key=true), " + - "attr INT" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1'], " + - " partition_by_hash_buckets = 2" + - ")"); - - assertUpdate("INSERT INTO test_grouped_execution VALUES (0, 0, 0), (0, 1, 1), (1, 0, 1)", 3); - assertQuery( - getSession(), - "SELECT key1, COUNT(1) FROM test_grouped_execution GROUP BY key1", - "VALUES (0, 2), (1, 1)", - assertRemoteExchangesCount(1)); - - assertUpdate("DROP TABLE test_grouped_execution"); - } - - @Test - public void testGroupedExecutionMultiLevelPartitioning() - { - assertUpdate("CREATE TABLE IF NOT EXISTS test_grouped_execution_mtlvl (" + - "key1 BIGINT WITH (primary_key=true)," + - "key2 BIGINT WITH (primary_key=true)," + - "key3 BIGINT WITH (primary_key=true)," + - "key4 BIGINT WITH (primary_key=true)," + - "attr1 BIGINT" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1', 'key2']," + - " partition_by_hash_buckets = 2," + - " partition_by_second_hash_columns = ARRAY['key3']," + - " partition_by_second_hash_buckets = 2" + - ")"); - - assertUpdate("INSERT INTO test_grouped_execution_mtlvl VALUES (0, 0, 0, 0, 0), (0, 0, 0, 1, 1), (1, 1, 1, 0, 0), (1, 1, 1, 1, 1)", 4); - assertQuery( - getSession(), - "SELECT key1, key2, key3, COUNT(1) FROM test_grouped_execution_mtlvl GROUP BY key1, key2, key3", - "VALUES (0, 0, 0, 2), (1, 1, 1, 2)", - assertRemoteExchangesCount(1)); - - assertUpdate("DROP TABLE test_grouped_execution_mtlvl"); - } - - @Test - public void testGroupedExecutionMultiLevelCombinedPartitioning() - { - assertUpdate("CREATE TABLE test_grouped_execution_hash_range (" + - "key1 BIGINT WITH (primary_key=true)," + - "key2 BIGINT WITH (primary_key=true)," + - "key3 BIGINT WITH (primary_key=true)," + - "key4 BIGINT WITH (primary_key=true)," + - "attr1 BIGINT" + - ") WITH (" + - " partition_by_hash_columns = ARRAY['key1']," + - " partition_by_hash_buckets = 2," + - " partition_by_second_hash_columns = ARRAY['key2']," + - " partition_by_second_hash_buckets = 3," + - " partition_by_range_columns = ARRAY['key3']," + - " range_partitions = '[{\"lower\": null, \"upper\": \"4\"}, {\"lower\": \"4\", \"upper\": \"9\"}, {\"lower\": \"9\", \"upper\": null}]'" + - ")"); - - assertUpdate("INSERT INTO test_grouped_execution_hash_range VALUES (0, 0, 0, 0, 0), (0, 0, 9, 0, 9), (0, 0, 9, 1, 0), (1, 1, 0, 0, 1), (1, 1, 9, 0, 2)", 5); - assertQuery( - getSession(), - "SELECT key1, key2, key3, COUNT(1) FROM test_grouped_execution_hash_range GROUP BY key1, key2, key3", - "VALUES (0, 0, 0, 1), (0, 0, 9, 2), (1, 1, 0, 1), (1, 1, 9, 1)", - assertRemoteExchangesCount(1)); - - assertUpdate("DROP TABLE test_grouped_execution_hash_range"); - } - - private Consumer assertRemoteExchangesCount(int expectedRemoteExchangesCount) - { - return plan -> { - int actualRemoteExchangesCount = searchFrom(plan.getRoot()) - .where(node -> node instanceof ExchangeNode && ((ExchangeNode) node).getScope() == ExchangeNode.Scope.REMOTE) - .findAll() - .size(); - if (actualRemoteExchangesCount != expectedRemoteExchangesCount) { - Session session = getSession(); - Metadata metadata = getDistributedQueryRunner().getCoordinator().getMetadata(); - FunctionManager functionManager = getDistributedQueryRunner().getCoordinator().getFunctionManager(); - String formattedPlan = textLogicalPlan(plan.getRoot(), plan.getTypes(), metadata, functionManager, StatsAndCosts.empty(), session, 0, false); - throw new AssertionError(format( - "Expected %s remote exchanges but found %s. Actual plan is:\n%s]", - expectedRemoteExchangesCount, - actualRemoteExchangesCount, - formattedPlan)); - } - }; - } -} diff --git a/plugin/trino-local-file/src/main/java/io/trino/plugin/localfile/LocalFileSplitManager.java b/plugin/trino-local-file/src/main/java/io/trino/plugin/localfile/LocalFileSplitManager.java index df82bf0dff8c..e40a6efb698c 100644 --- a/plugin/trino-local-file/src/main/java/io/trino/plugin/localfile/LocalFileSplitManager.java +++ b/plugin/trino-local-file/src/main/java/io/trino/plugin/localfile/LocalFileSplitManager.java @@ -47,7 +47,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-memory/src/main/java/io/trino/plugin/memory/MemorySplitManager.java b/plugin/trino-memory/src/main/java/io/trino/plugin/memory/MemorySplitManager.java index 0bac1b6e57f5..b3607001f49b 100644 --- a/plugin/trino-memory/src/main/java/io/trino/plugin/memory/MemorySplitManager.java +++ b/plugin/trino-memory/src/main/java/io/trino/plugin/memory/MemorySplitManager.java @@ -47,7 +47,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle handle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java index cde825a6b54c..d958ba25b135 100644 --- a/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java +++ b/plugin/trino-mongodb/src/main/java/io/trino/plugin/mongodb/MongoSplitManager.java @@ -44,7 +44,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java index 738b7e368b3b..9f8885740743 100644 --- a/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java +++ b/plugin/trino-phoenix5/src/main/java/io/trino/plugin/phoenix5/PhoenixSplitManager.java @@ -74,7 +74,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotSplitManager.java b/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotSplitManager.java index 543e2ab78e21..be2fbef644bb 100755 --- a/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotSplitManager.java +++ b/plugin/trino-pinot/src/main/java/io/trino/plugin/pinot/PinotSplitManager.java @@ -163,7 +163,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transactionHandle, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotSplitManager.java b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotSplitManager.java index a4939f26b46a..cb8821447370 100755 --- a/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotSplitManager.java +++ b/plugin/trino-pinot/src/test/java/io/trino/plugin/pinot/TestPinotSplitManager.java @@ -34,8 +34,6 @@ import static io.trino.plugin.pinot.PinotSplit.SplitType.BROKER; import static io.trino.plugin.pinot.PinotSplit.SplitType.SEGMENT; import static io.trino.plugin.pinot.query.DynamicTableBuilder.buildFromPql; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static java.lang.String.format; import static java.util.stream.Collectors.toList; @@ -126,10 +124,10 @@ public static ConnectorSession createSessionWithNumSplits(int numSegmentsPerSpli private List getSplitsHelper(PinotTableHandle pinotTable, int numSegmentsPerSplit, boolean forbidSegmentQueries) { ConnectorSession session = createSessionWithNumSplits(numSegmentsPerSplit, forbidSegmentQueries, pinotConfig); - ConnectorSplitSource splitSource = pinotSplitManager.getSplits(null, session, pinotTable, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + ConnectorSplitSource splitSource = pinotSplitManager.getSplits(null, session, pinotTable, DynamicFilter.EMPTY, Constraint.alwaysTrue()); List splits = new ArrayList<>(); while (!splitSource.isFinished()) { - splits.addAll(getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, 1000)).getSplits().stream().map(s -> (PinotSplit) s).collect(toList())); + splits.addAll(getFutureValue(splitSource.getNextBatch(1000)).getSplits().stream().map(s -> (PinotSplit) s).collect(toList())); } return splits; diff --git a/plugin/trino-prometheus/src/main/java/io/trino/plugin/prometheus/PrometheusSplitManager.java b/plugin/trino-prometheus/src/main/java/io/trino/plugin/prometheus/PrometheusSplitManager.java index 0e6aae720f36..34cca2e3379f 100644 --- a/plugin/trino-prometheus/src/main/java/io/trino/plugin/prometheus/PrometheusSplitManager.java +++ b/plugin/trino-prometheus/src/main/java/io/trino/plugin/prometheus/PrometheusSplitManager.java @@ -81,7 +81,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle connectorTableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java index 96d6b16f17aa..7d6136d3584f 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusIntegration.java @@ -27,7 +27,6 @@ import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusClient; import static io.trino.plugin.prometheus.PrometheusQueryRunner.createPrometheusQueryRunner; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static java.util.concurrent.TimeUnit.DAYS; import static java.util.concurrent.TimeUnit.SECONDS; import static org.assertj.core.api.Assertions.assertThat; @@ -120,10 +119,9 @@ public void testCorrectNumberOfSplitsCreated() null, null, new PrometheusTableHandle("default", table.getName()), - null, (DynamicFilter) null, Constraint.alwaysTrue()); - int numSplits = splits.getNextBatch(NOT_PARTITIONED, NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits().size(); + int numSplits = splits.getNextBatch(NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits().size(); assertEquals(numSplits, config.getMaxQueryRangeDuration().getValue(TimeUnit.SECONDS) / config.getQueryChunkSizeDuration().getValue(TimeUnit.SECONDS), 0.001); } diff --git a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java index 4f77a757dc90..f7ba4b8b1cdd 100644 --- a/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java +++ b/plugin/trino-prometheus/src/test/java/io/trino/plugin/prometheus/TestPrometheusSplit.java @@ -51,7 +51,6 @@ import static io.trino.plugin.prometheus.PrometheusClock.fixedClockAt; import static io.trino.plugin.prometheus.PrometheusSplitManager.OFFSET_MILLIS; import static io.trino.plugin.prometheus.PrometheusSplitManager.decimalSecondString; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; @@ -126,10 +125,9 @@ public void testQueryWithTableNameNeedingURLEncodeInSplits() null, null, new PrometheusTableHandle("default", table.getName()), - null, (DynamicFilter) null, Constraint.alwaysTrue()); - PrometheusSplit split = (PrometheusSplit) splits.getNextBatch(NOT_PARTITIONED, 1).getNow(null).getSplits().get(0); + PrometheusSplit split = (PrometheusSplit) splits.getNextBatch(1).getNow(null).getSplits().get(0); String queryInSplit = URI.create(split.getUri()).getQuery(); String timeShouldBe = decimalSecondString(now.toEpochMilli() - config.getMaxQueryRangeDuration().toMillis() + @@ -153,10 +151,9 @@ public void testQueryDividedIntoSplitsFirstSplitHasRightTime() null, null, new PrometheusTableHandle("default", table.getName()), - null, (DynamicFilter) null, Constraint.alwaysTrue()); - PrometheusSplit split = (PrometheusSplit) splits.getNextBatch(NOT_PARTITIONED, 1).getNow(null).getSplits().get(0); + PrometheusSplit split = (PrometheusSplit) splits.getNextBatch(1).getNow(null).getSplits().get(0); String queryInSplit = URI.create(split.getUri()).getQuery(); String timeShouldBe = decimalSecondString(now.toEpochMilli() - config.getMaxQueryRangeDuration().toMillis() + @@ -180,10 +177,9 @@ public void testQueryDividedIntoSplitsLastSplitHasRightTime() null, null, new PrometheusTableHandle("default", table.getName()), - null, (DynamicFilter) null, Constraint.alwaysTrue()); - List splits = splitsMaybe.getNextBatch(NOT_PARTITIONED, NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits(); + List splits = splitsMaybe.getNextBatch(NUMBER_MORE_THAN_EXPECTED_NUMBER_SPLITS).getNow(null).getSplits(); int lastSplitIndex = splits.size() - 1; PrometheusSplit lastSplit = (PrometheusSplit) splits.get(lastSplitIndex); String queryInSplit = URI.create(lastSplit.getUri()).getQuery(); @@ -206,12 +202,11 @@ public void testQueryDividedIntoSplitsShouldHaveCorrectSpacingBetweenTimes() null, null, new PrometheusTableHandle("default", table.getName()), - null, (DynamicFilter) null, Constraint.alwaysTrue()); - PrometheusSplit split1 = (PrometheusSplit) splits.getNextBatch(NOT_PARTITIONED, 1).getNow(null).getSplits().get(0); + PrometheusSplit split1 = (PrometheusSplit) splits.getNextBatch(1).getNow(null).getSplits().get(0); Map paramsMap1 = parse(URI.create(split1.getUri()), StandardCharsets.UTF_8).stream().collect(Collectors.toMap(NameValuePair::getName, NameValuePair::getValue)); - PrometheusSplit split2 = (PrometheusSplit) splits.getNextBatch(NOT_PARTITIONED, 1).getNow(null).getSplits().get(0); + PrometheusSplit split2 = (PrometheusSplit) splits.getNextBatch(1).getNow(null).getSplits().get(0); Map paramsMap2 = parse(URI.create(split2.getUri()), StandardCharsets.UTF_8).stream().collect(Collectors.toMap(NameValuePair::getName, NameValuePair::getValue)); assertEquals(paramsMap1.get("query"), "up[1d]"); assertEquals(paramsMap2.get("query"), "up[1d]"); diff --git a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/RaptorSplitManager.java b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/RaptorSplitManager.java index bda33d6a1e87..9a744875ae41 100644 --- a/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/RaptorSplitManager.java +++ b/plugin/trino-raptor-legacy/src/main/java/io/trino/plugin/raptor/legacy/RaptorSplitManager.java @@ -23,7 +23,6 @@ import io.trino.spi.HostAddress; import io.trino.spi.Node; import io.trino.spi.TrinoException; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitManager; @@ -97,7 +96,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle handle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { @@ -163,7 +161,7 @@ public RaptorSplitSource( } @Override - public synchronized CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public synchronized CompletableFuture getNextBatch(int maxSize) { checkState((future == null) || future.isDone(), "previous batch not completed"); future = supplyAsync(batchSupplier(maxSize), executor); diff --git a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/metadata/TestRaptorSplitManager.java b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/metadata/TestRaptorSplitManager.java index 43ebef9e7d74..673a833c1161 100644 --- a/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/metadata/TestRaptorSplitManager.java +++ b/plugin/trino-raptor-legacy/src/test/java/io/trino/plugin/raptor/legacy/metadata/TestRaptorSplitManager.java @@ -61,8 +61,6 @@ import static io.trino.plugin.raptor.legacy.metadata.DatabaseShardManager.shardIndexTable; import static io.trino.plugin.raptor.legacy.metadata.SchemaDaoUtil.createTablesWithRetry; import static io.trino.plugin.raptor.legacy.metadata.TestDatabaseShardManager.shardInfo; -import static io.trino.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.VarcharType.createVarcharType; import static io.trino.testing.TestingConnectorSession.SESSION; import static java.lang.String.format; @@ -195,11 +193,11 @@ private void deleteShardNodes() private static ConnectorSplitSource getSplits(RaptorSplitManager splitManager, ConnectorTableHandle table) { ConnectorTransactionHandle transaction = new RaptorTransactionHandle(); - return splitManager.getSplits(transaction, SESSION, table, UNGROUPED_SCHEDULING, DynamicFilter.EMPTY, Constraint.alwaysTrue()); + return splitManager.getSplits(transaction, SESSION, table, DynamicFilter.EMPTY, Constraint.alwaysTrue()); } private static List getSplits(ConnectorSplitSource source, int maxSize) { - return getFutureValue(source.getNextBatch(NOT_PARTITIONED, maxSize)).getSplits(); + return getFutureValue(source.getNextBatch(maxSize)).getSplits(); } } diff --git a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisSplitManager.java b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisSplitManager.java index 9c09118b0f16..bd280eaa3dea 100644 --- a/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisSplitManager.java +++ b/plugin/trino-redis/src/main/java/io/trino/plugin/redis/RedisSplitManager.java @@ -61,7 +61,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-thrift/src/main/java/io/trino/plugin/thrift/ThriftSplitManager.java b/plugin/trino-thrift/src/main/java/io/trino/plugin/thrift/ThriftSplitManager.java index 36f684ec211c..38895ca4ccf9 100644 --- a/plugin/trino-thrift/src/main/java/io/trino/plugin/thrift/ThriftSplitManager.java +++ b/plugin/trino-thrift/src/main/java/io/trino/plugin/thrift/ThriftSplitManager.java @@ -27,7 +27,6 @@ import io.trino.plugin.thrift.api.TrinoThriftTupleDomain; import io.trino.spi.HostAddress; import io.trino.spi.connector.ColumnHandle; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitManager; @@ -75,7 +74,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { @@ -131,7 +129,7 @@ public ThriftSplitSource( * It can be called by multiple threads, but only if the previous call finished. */ @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { checkState(future.get() == null || future.get().isDone(), "previous batch not completed"); checkState(hasMoreData.get(), "this method cannot be invoked when there's no more data"); diff --git a/plugin/trino-tpcds/src/main/java/io/trino/plugin/tpcds/TpcdsSplitManager.java b/plugin/trino-tpcds/src/main/java/io/trino/plugin/tpcds/TpcdsSplitManager.java index 6a35a93f0b34..febb6ce89bc3 100644 --- a/plugin/trino-tpcds/src/main/java/io/trino/plugin/tpcds/TpcdsSplitManager.java +++ b/plugin/trino-tpcds/src/main/java/io/trino/plugin/tpcds/TpcdsSplitManager.java @@ -57,7 +57,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle tableHandle, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { diff --git a/plugin/trino-tpch/src/main/java/io/trino/plugin/tpch/TpchSplitManager.java b/plugin/trino-tpch/src/main/java/io/trino/plugin/tpch/TpchSplitManager.java index 2eb2da02e78e..146ad44c389b 100644 --- a/plugin/trino-tpch/src/main/java/io/trino/plugin/tpch/TpchSplitManager.java +++ b/plugin/trino-tpch/src/main/java/io/trino/plugin/tpch/TpchSplitManager.java @@ -51,7 +51,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { 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 bc6416efceba..1dfb8bcde37f 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 @@ -20,7 +20,6 @@ import io.airlift.stats.TestingGcMonitor; import io.airlift.units.DataSize; import io.trino.Session; -import io.trino.execution.Lifespan; import io.trino.execution.StageId; import io.trino.execution.TaskId; import io.trino.execution.TaskStateMachine; @@ -82,10 +81,8 @@ 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.spi.connector.ConnectorSplitManager.SplitSchedulingStrategy.UNGROUPED_SCHEDULING; import static io.trino.spi.connector.Constraint.alwaysTrue; import static io.trino.spi.connector.DynamicFilter.EMPTY; -import static io.trino.spi.connector.NotPartitionedPartitionHandle.NOT_PARTITIONED; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes; import static io.trino.sql.planner.TypeAnalyzer.createTestingTypeAnalyzer; @@ -212,7 +209,7 @@ public OperatorFactory duplicate() private Split getLocalQuerySplit(Session session, TableHandle handle) { - SplitSource splitSource = localQueryRunner.getSplitManager().getSplits(session, handle, UNGROUPED_SCHEDULING, EMPTY, alwaysTrue()); + SplitSource splitSource = localQueryRunner.getSplitManager().getSplits(session, handle, EMPTY, alwaysTrue()); List splits = new ArrayList<>(); while (!splitSource.isFinished()) { splits.addAll(getNextBatch(splitSource)); @@ -223,7 +220,7 @@ private Split getLocalQuerySplit(Session session, TableHandle handle) private static List getNextBatch(SplitSource splitSource) { - return getFutureValue(splitSource.getNextBatch(NOT_PARTITIONED, Lifespan.taskWide(), 1000)).getSplits(); + return getFutureValue(splitSource.getNextBatch(1000)).getSplits(); } protected final OperatorFactory createHashProjectOperator(int operatorId, PlanNodeId planNodeId, List types) diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractSimpleOperatorBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractSimpleOperatorBenchmark.java index af0052dd06ec..d3e7453131fd 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractSimpleOperatorBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/AbstractSimpleOperatorBenchmark.java @@ -27,8 +27,6 @@ import java.util.List; import java.util.OptionalInt; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; - public abstract class AbstractSimpleOperatorBenchmark extends AbstractOperatorBenchmark { @@ -49,7 +47,7 @@ protected DriverFactory createDriverFactory() operatorFactories.add(new NullOutputOperatorFactory(999, new PlanNodeId("test"))); - return new DriverFactory(0, true, true, operatorFactories, OptionalInt.empty(), UNGROUPED_EXECUTION); + return new DriverFactory(0, true, true, operatorFactories, OptionalInt.empty()); } @Override diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java index d95f894b03cd..b01d4f399527 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildAndJoinBenchmark.java @@ -43,7 +43,6 @@ import static io.trino.benchmark.BenchmarkQueryRunner.createLocalQueryRunner; import static io.trino.benchmark.BenchmarkQueryRunner.createLocalQueryRunnerHashEnabled; import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spiller.PartitioningSpillerFactory.unsupportedPartitioningSpillerFactory; import static io.trino.testing.TestingSession.testSessionBuilder; @@ -125,7 +124,7 @@ protected List createDrivers(TaskContext taskContext) SingleStreamSpillerFactory.unsupportedSingleStreamSpillerFactory(), incrementalLoadFactorHashArraySizeSupplier(session)); driversBuilder.add(hashBuilder); - DriverFactory hashBuildDriverFactory = new DriverFactory(0, true, false, driversBuilder.build(), OptionalInt.empty(), UNGROUPED_EXECUTION); + DriverFactory hashBuildDriverFactory = new DriverFactory(0, true, false, driversBuilder.build(), OptionalInt.empty()); // join ImmutableList.Builder joinDriversBuilder = ImmutableList.builder(); @@ -157,7 +156,7 @@ protected List createDrivers(TaskContext taskContext) blockTypeOperators); joinDriversBuilder.add(joinOperator); joinDriversBuilder.add(new NullOutputOperatorFactory(3, new PlanNodeId("test"))); - DriverFactory joinDriverFactory = new DriverFactory(1, true, true, joinDriversBuilder.build(), OptionalInt.empty(), UNGROUPED_EXECUTION); + DriverFactory joinDriverFactory = new DriverFactory(1, true, true, joinDriversBuilder.build(), OptionalInt.empty()); Driver hashBuildDriver = hashBuildDriverFactory.createDriver(taskContext.addPipelineContext(0, true, false, false).addDriverContext()); hashBuildDriverFactory.noMoreDrivers(); diff --git a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java index 6128958b166e..7a2e7a12f58d 100644 --- a/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java +++ b/testing/trino-benchmark/src/main/java/io/trino/benchmark/HashBuildBenchmark.java @@ -41,7 +41,6 @@ import static com.google.common.collect.ImmutableList.toImmutableList; import static io.trino.benchmark.BenchmarkQueryRunner.createLocalQueryRunner; import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spiller.PartitioningSpillerFactory.unsupportedPartitioningSpillerFactory; import static java.util.Objects.requireNonNull; @@ -95,7 +94,7 @@ protected List createDrivers(TaskContext taskContext) false, SingleStreamSpillerFactory.unsupportedSingleStreamSpillerFactory(), incrementalLoadFactorHashArraySizeSupplier(session)); - DriverFactory hashBuildDriverFactory = new DriverFactory(0, true, true, ImmutableList.of(ordersTableScan, hashBuilder), OptionalInt.empty(), UNGROUPED_EXECUTION); + DriverFactory hashBuildDriverFactory = new DriverFactory(0, true, true, ImmutableList.of(ordersTableScan, hashBuilder), OptionalInt.empty()); // empty join so build finishes ImmutableList.Builder joinDriversBuilder = ImmutableList.builder(); @@ -116,7 +115,7 @@ protected List createDrivers(TaskContext taskContext) blockTypeOperators); joinDriversBuilder.add(joinOperator); joinDriversBuilder.add(new NullOutputOperatorFactory(3, new PlanNodeId("test"))); - DriverFactory joinDriverFactory = new DriverFactory(1, true, true, joinDriversBuilder.build(), OptionalInt.empty(), UNGROUPED_EXECUTION); + DriverFactory joinDriverFactory = new DriverFactory(1, true, true, joinDriversBuilder.build(), OptionalInt.empty()); Driver hashBuildDriver = hashBuildDriverFactory.createDriver(taskContext.addPipelineContext(0, true, true, false).addDriverContext()); hashBuildDriverFactory.noMoreDrivers(); 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 a906834775ed..05ab7aedea1c 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 @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.primitives.Ints; -import io.trino.execution.Lifespan; import io.trino.operator.Driver; import io.trino.operator.DriverContext; import io.trino.operator.DriverFactory; @@ -46,7 +45,6 @@ import static io.trino.benchmark.BenchmarkQueryRunner.createLocalQueryRunner; import static io.trino.execution.executor.PrioritizedSplitRunner.SPLIT_RUN_QUANTA; import static io.trino.operator.HashArraySizeSupplier.incrementalLoadFactorHashArraySizeSupplier; -import static io.trino.operator.PipelineExecutionStrategy.UNGROUPED_EXECUTION; import static io.trino.spiller.PartitioningSpillerFactory.unsupportedPartitioningSpillerFactory; import static java.util.Objects.requireNonNull; @@ -107,7 +105,7 @@ protected List createDrivers(TaskContext taskContext) incrementalLoadFactorHashArraySizeSupplier(session)); DriverContext driverContext = taskContext.addPipelineContext(0, false, false, false).addDriverContext(); - DriverFactory buildDriverFactory = new DriverFactory(0, false, false, ImmutableList.of(ordersTableScan, hashBuilder), OptionalInt.empty(), UNGROUPED_EXECUTION); + DriverFactory buildDriverFactory = new DriverFactory(0, false, false, ImmutableList.of(ordersTableScan, hashBuilder), OptionalInt.empty()); List lineItemTypes = getColumnTypes("lineitem", "orderkey", "quantity"); OperatorFactory lineItemTableScan = createTableScanOperator(0, new PlanNodeId("test"), "lineitem", "orderkey", "quantity"); @@ -126,10 +124,10 @@ protected List createDrivers(TaskContext taskContext) unsupportedPartitioningSpillerFactory(), blockTypeOperators); NullOutputOperatorFactory output = new NullOutputOperatorFactory(2, new PlanNodeId("test")); - this.probeDriverFactory = new DriverFactory(1, true, true, ImmutableList.of(lineItemTableScan, joinOperator, output), OptionalInt.empty(), UNGROUPED_EXECUTION); + this.probeDriverFactory = new DriverFactory(1, true, true, ImmutableList.of(lineItemTableScan, joinOperator, output), OptionalInt.empty()); Driver driver = buildDriverFactory.createDriver(driverContext); - Future lookupSourceProvider = lookupSourceFactoryManager.getJoinBridge(Lifespan.taskWide()).createLookupSourceProvider(); + Future lookupSourceProvider = lookupSourceFactoryManager.getJoinBridge().createLookupSourceProvider(); while (!lookupSourceProvider.isDone()) { driver.processForDuration(SPLIT_RUN_QUANTA); } diff --git a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionConnectorTest.java b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionConnectorTest.java index e93718e4c0f6..1400039a6375 100644 --- a/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionConnectorTest.java +++ b/testing/trino-faulttolerant-tests/src/test/java/io/trino/faulttolerant/hive/TestHiveFaultTolerantExecutionConnectorTest.java @@ -47,12 +47,6 @@ protected QueryRunner createQueryRunner() }); } - @Override - public void testGroupedExecution() - { - // grouped execution is not supported (and not needed) with batch execution enabled - } - @Override public void testScaleWriters() { diff --git a/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java b/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java index 4b60f887656e..8d5b9f2f8b72 100644 --- a/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java +++ b/testing/trino-tests/src/test/java/io/trino/execution/TestCoordinatorDynamicFiltering.java @@ -28,7 +28,6 @@ import io.trino.spi.connector.ConnectorPageSinkProvider; import io.trino.spi.connector.ConnectorPageSource; import io.trino.spi.connector.ConnectorPageSourceProvider; -import io.trino.spi.connector.ConnectorPartitionHandle; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.connector.ConnectorSplitManager; @@ -462,7 +461,6 @@ public ConnectorSplitSource getSplits( ConnectorTransactionHandle transaction, ConnectorSession session, ConnectorTableHandle table, - SplitSchedulingStrategy splitSchedulingStrategy, DynamicFilter dynamicFilter, Constraint constraint) { @@ -474,7 +472,7 @@ public ConnectorSplitSource getSplits( return new ConnectorSplitSource() { @Override - public CompletableFuture getNextBatch(ConnectorPartitionHandle partitionHandle, int maxSize) + public CompletableFuture getNextBatch(int maxSize) { CompletableFuture blocked = dynamicFilter.isBlocked();