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 59be671b8ce1..e5fc461397ac 100644 --- a/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java +++ b/core/trino-main/src/main/java/io/trino/SystemSessionProperties.java @@ -143,6 +143,9 @@ public final class SystemSessionProperties public static final String ENABLE_LARGE_DYNAMIC_FILTERS = "enable_large_dynamic_filters"; public static final String ENABLE_DYNAMIC_ROW_FILTERING = "enable_dynamic_row_filtering"; public static final String DYNAMIC_ROW_FILTERING_SELECTIVITY_THRESHOLD = "dynamic_row_filtering_selectivity_threshold"; + public static final String PREFERRED_DYNAMIC_FILTER_WAIT_TIMEOUT = "preferred_dynamic_filter_wait_timeout"; + public static final String AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT = "awaited_dynamic_filter_max_row_count"; + public static final String AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT = "awaited_dynamic_filter_max_ndv_count"; public static final String QUERY_MAX_MEMORY_PER_NODE = "query_max_memory_per_node"; public static final String IGNORE_DOWNSTREAM_PREFERENCES = "ignore_downstream_preferences"; public static final String FILTERING_SEMI_JOIN_TO_INNER = "rewrite_filtering_semi_join_to_inner_join"; @@ -702,6 +705,23 @@ public SystemSessionProperties( } }, false), + durationProperty( + PREFERRED_DYNAMIC_FILTER_WAIT_TIMEOUT, + "Maximum preferred time to wait for awaitable dynamic filter before table scan is started", + dynamicFilterConfig.getPreferredDynamicFilterWaitTimeout(), + false), + longProperty( + AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT, + "Maximum number of rows for dynamic filter to be awaitable", + dynamicFilterConfig.getAwaitedDynamicFilterMaxRowCount(), + value -> validateNonNegativeLongValue(value, AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT), + false), + longProperty( + AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT, + "Maximum number of distinct values for dynamic filter to be awaitable", + dynamicFilterConfig.getAwaitedDynamicFilterMaxNdvCount(), + value -> validateNonNegativeLongValue(value, AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT), + false), dataSizeProperty( QUERY_MAX_MEMORY_PER_NODE, "Maximum amount of memory a query can use per node", @@ -1638,6 +1658,21 @@ public static double getDynamicRowFilterSelectivityThreshold(Session session) return session.getSystemProperty(DYNAMIC_ROW_FILTERING_SELECTIVITY_THRESHOLD, Double.class); } + public static Duration getPreferredDynamicFilterWaitTimeout(Session session) + { + return session.getSystemProperty(PREFERRED_DYNAMIC_FILTER_WAIT_TIMEOUT, Duration.class); + } + + public static long getAwaitedDynamicFilterMaxRowCount(Session session) + { + return session.getSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT, Long.class); + } + + public static long getAwaitedDynamicFilterMaxNdvCount(Session session) + { + return session.getSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT, Long.class); + } + public static DataSize getQueryMaxMemoryPerNode(Session session) { return session.getSystemProperty(QUERY_MAX_MEMORY_PER_NODE, DataSize.class); diff --git a/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java b/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java index 8ac74b3c878a..fc54b71bdc80 100644 --- a/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java +++ b/core/trino-main/src/main/java/io/trino/execution/DynamicFilterConfig.java @@ -18,7 +18,9 @@ import io.airlift.configuration.DefunctConfig; import io.airlift.configuration.LegacyConfig; import io.airlift.units.DataSize; +import io.airlift.units.Duration; import io.airlift.units.MaxDataSize; +import io.airlift.units.MinDuration; import jakarta.validation.constraints.DecimalMax; import jakarta.validation.constraints.DecimalMin; import jakarta.validation.constraints.Min; @@ -26,6 +28,7 @@ import static io.airlift.units.DataSize.Unit.KILOBYTE; import static io.airlift.units.DataSize.Unit.MEGABYTE; +import static java.util.concurrent.TimeUnit.SECONDS; @DefunctConfig({ "dynamic-filtering-max-per-driver-row-count", @@ -45,6 +48,9 @@ public class DynamicFilterConfig private boolean enableDynamicRowFiltering = true; private double dynamicRowFilterSelectivityThreshold = 0.7; + private Duration preferredDynamicFilterWaitTimeout = new Duration(20, SECONDS); + private long awaitedDynamicFilterMaxRowCount = 100_000; + private long awaitedDynamicFilterMaxNdvCount = 500; /* * dynamic-filtering.small.* and dynamic-filtering.large.* limits are applied when * collected over a not pre-partitioned source (when join distribution type is @@ -129,6 +135,48 @@ public DynamicFilterConfig setDynamicRowFilterSelectivityThreshold(double dynami return this; } + @MinDuration("0ms") + public Duration getPreferredDynamicFilterWaitTimeout() + { + return preferredDynamicFilterWaitTimeout; + } + + @Config("preferred-dynamic-filter.wait-timeout") + @ConfigDescription("Maximum preferred time to wait for awaitable dynamic filter before table scan is started") + public DynamicFilterConfig setPreferredDynamicFilterWaitTimeout(Duration dynamicFilteringWaitTimeout) + { + this.preferredDynamicFilterWaitTimeout = dynamicFilteringWaitTimeout; + return this; + } + + @Min(0) + public long getAwaitedDynamicFilterMaxRowCount() + { + return awaitedDynamicFilterMaxRowCount; + } + + @Config("awaited-dynamic-filter.max-row-count") + @ConfigDescription("Maximum number of rows for dynamic filter to be awaited") + public DynamicFilterConfig setAwaitedDynamicFilterMaxRowCount(long awaitedDynamicFilterMaxRowCount) + { + this.awaitedDynamicFilterMaxRowCount = awaitedDynamicFilterMaxRowCount; + return this; + } + + @Min(0) + public long getAwaitedDynamicFilterMaxNdvCount() + { + return awaitedDynamicFilterMaxNdvCount; + } + + @Config("awaited-dynamic-filter.max-ndv-count") + @ConfigDescription("Maximum number of distinct values for dynamic filter to be awaited") + public DynamicFilterConfig setAwaitedDynamicFilterMaxNdvCount(long awaitedDynamicFilterMaxNdvCount) + { + this.awaitedDynamicFilterMaxNdvCount = awaitedDynamicFilterMaxNdvCount; + return this; + } + @Min(0) public int getSmallMaxDistinctValuesPerDriver() { diff --git a/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java b/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java index d24691b1313a..9abd8fe5ec63 100644 --- a/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java +++ b/core/trino-main/src/main/java/io/trino/server/DynamicFilterService.java @@ -62,6 +62,7 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; import java.util.Queue; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -70,6 +71,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.Function; import static com.google.common.base.Functions.identity; import static com.google.common.base.MoreObjects.toStringHelper; @@ -286,10 +288,10 @@ public DynamicFilter createDynamicFilter( return EMPTY; } - List> lazyDynamicFilterFutures = dynamicFilters.stream() - .map(context.getLazyDynamicFilters()::get) - .filter(Objects::nonNull) - .collect(toImmutableList()); + Map> lazyDynamicFilterFutures = dynamicFilters.stream() + .filter(dynamicFilterId -> context.getLazyDynamicFilters().containsKey(dynamicFilterId)) + .collect(toImmutableMap(Function.identity(), dynamicFilterId -> context.getLazyDynamicFilters().get(dynamicFilterId))); + AtomicReference currentDynamicFilter = new AtomicReference<>(new CurrentDynamicFilter(0, TupleDomain.all())); Set columnsCovered = symbolsMap.values().stream() @@ -298,6 +300,22 @@ public DynamicFilter createDynamicFilter( .map(probeSymbol -> requireNonNull(columnHandles.get(probeSymbol), () -> "Missing probe column for " + probeSymbol)) .collect(toImmutableSet()); + Map dynamicFilterTimeouts = dynamicFilterDescriptors.stream() + .collect(toImmutableMap( + DynamicFilters.Descriptor::getId, + DynamicFilters.Descriptor::getPreferredTimeout, + (timeout1, timeout2) -> { + if (timeout1.isPresent() && timeout2.isPresent()) { + return OptionalLong.of(Long.max(timeout1.getAsLong(), timeout2.getAsLong())); + } + else if (timeout1.isPresent()) { + return timeout1; + } + else { + return timeout2; + } + })); + return new DynamicFilter() { @Override @@ -310,7 +328,7 @@ public Set getColumnsCovered() public CompletableFuture isBlocked() { // wait for any of the requested dynamic filter domains to be completed - List> undoneFutures = lazyDynamicFilterFutures.stream() + List> undoneFutures = lazyDynamicFilterFutures.values().stream() .filter(future -> !future.isDone()) .collect(toImmutableList()); @@ -331,7 +349,7 @@ public boolean isComplete() @Override public boolean isAwaitable() { - return lazyDynamicFilterFutures.stream() + return lazyDynamicFilterFutures.values().stream() .anyMatch(future -> !future.isDone()); } @@ -363,6 +381,33 @@ public TupleDomain getCurrentPredicate() currentDynamicFilter.set(new CurrentDynamicFilter(completedDynamicFilters.size(), dynamicFilter)); return dynamicFilter; } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + long longestPreferredTimeout = -1; + boolean unestimatedDynamicFilter = false; + + for (Map.Entry> lazyDynamicFilterFuture : lazyDynamicFilterFutures.entrySet()) { + if (!lazyDynamicFilterFuture.getValue().isDone()) { + OptionalLong preferredTimeout = dynamicFilterTimeouts.get(lazyDynamicFilterFuture.getKey()); + if (preferredTimeout != null) { + if (preferredTimeout.isPresent()) { + longestPreferredTimeout = Long.max(preferredTimeout.getAsLong(), longestPreferredTimeout); + } + else { + unestimatedDynamicFilter = true; + } + } + } + } + + if (longestPreferredTimeout == -1 || (unestimatedDynamicFilter && longestPreferredTimeout == 0)) { + // at least one unestimated dynamic filter timeout was found and rest of DF are not awaitable + return OptionalLong.empty(); + } + return OptionalLong.of(longestPreferredTimeout); + } }; } diff --git a/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java b/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java index 15b2395b9c29..df919458f683 100644 --- a/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java +++ b/core/trino-main/src/main/java/io/trino/sql/DynamicFilters.java @@ -22,12 +22,14 @@ import io.trino.spi.function.CatalogSchemaFunctionName; import io.trino.spi.function.IsNull; import io.trino.spi.function.ScalarFunction; +import io.trino.spi.function.SqlNullable; import io.trino.spi.function.SqlType; import io.trino.spi.function.TypeParameter; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.Range; import io.trino.spi.predicate.ValueSet; import io.trino.spi.type.BooleanType; +import io.trino.spi.type.IntegerType; import io.trino.spi.type.Type; import io.trino.spi.type.VarcharType; import io.trino.sql.ir.Call; @@ -43,6 +45,7 @@ import java.util.List; import java.util.Objects; import java.util.Optional; +import java.util.OptionalLong; import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkArgument; @@ -50,11 +53,13 @@ import static com.google.common.collect.ImmutableListMultimap.toImmutableListMultimap; import static io.trino.metadata.GlobalFunctionCatalog.builtinFunctionName; import static io.trino.spi.type.StandardTypes.BOOLEAN; +import static io.trino.spi.type.StandardTypes.INTEGER; import static io.trino.spi.type.StandardTypes.VARCHAR; import static io.trino.sql.ir.Booleans.FALSE; import static io.trino.sql.ir.Booleans.TRUE; import static io.trino.sql.ir.Comparison.Operator.EQUAL; import static io.trino.sql.ir.IrUtils.extractConjuncts; +import static java.lang.String.format; import static java.util.Objects.requireNonNull; public final class DynamicFilters @@ -79,12 +84,27 @@ public static Expression createDynamicFilterExpression( Comparison.Operator operator, boolean nullAllowed) { + return createDynamicFilterExpression(metadata, id, inputType, input, operator, nullAllowed, Optional.empty()); + } + + @VisibleForTesting + public static Expression createDynamicFilterExpression( + Metadata metadata, + DynamicFilterId id, + Type inputType, + Expression input, + Comparison.Operator operator, + boolean nullAllowed, + Optional minDynamicFilterTimeout) + { + Expression timeoutExpression = new Constant(IntegerType.INTEGER, minDynamicFilterTimeout.orElse(null)); return BuiltinFunctionCallBuilder.resolve(metadata) .setName(nullAllowed ? NullableFunction.NAME : Function.NAME) .addArgument(inputType, input) .addArgument(new Constant(VarcharType.VARCHAR, Slices.utf8Slice(operator.toString()))) .addArgument(new Constant(VarcharType.VARCHAR, Slices.utf8Slice(id.toString()))) .addArgument(BooleanType.BOOLEAN, nullAllowed ? TRUE : FALSE) + .addArgument(IntegerType.INTEGER, timeoutExpression) .build(); } @@ -123,10 +143,11 @@ public static Multimap extractSourceSymbols(List getDescriptor(Expression expression) } List arguments = call.arguments(); - checkArgument(arguments.size() == 4, "invalid arguments count: %s", arguments.size()); + checkArgument(arguments.size() == 5, "invalid arguments count: %s", arguments.size()); Expression probeSymbol = arguments.get(0); @@ -180,7 +216,23 @@ public static Optional getDescriptor(Expression expression) Expression nullAllowedExpression = arguments.get(3); checkArgument(nullAllowedExpression instanceof Constant literal && literal.type().equals(BooleanType.BOOLEAN), "nullAllowedExpression is expected to be a boolean constant: %s", nullAllowedExpression.getClass().getSimpleName()); boolean nullAllowed = (boolean) ((Constant) nullAllowedExpression).value(); - return Optional.of(new Descriptor(new DynamicFilterId(id), probeSymbol, operator, nullAllowed)); + + Expression timeoutExpression = arguments.get(4); + OptionalLong timeout; + if (timeoutExpression instanceof Constant timeoutConstant && timeoutConstant.type().getJavaType() == long.class) { + Long value = (Long) timeoutConstant.value(); + if (value == null) { + timeout = OptionalLong.empty(); + } + else { + timeout = OptionalLong.of(value); + } + } + else { + throw new IllegalArgumentException(format("timeout is expected to be integer constant: %s", timeoutExpression.getClass().getSimpleName())); + } + + return Optional.of(new Descriptor(new DynamicFilterId(id), probeSymbol, operator, nullAllowed, timeout)); } private static boolean isDynamicFilterFunction(Call call) @@ -221,19 +273,21 @@ public static final class Descriptor private final Expression input; private final Comparison.Operator operator; private final boolean nullAllowed; + private final OptionalLong preferredTimeout; - public Descriptor(DynamicFilterId id, Expression input, Comparison.Operator operator, boolean nullAllowed) + public Descriptor(DynamicFilterId id, Expression input, Comparison.Operator operator, boolean nullAllowed, OptionalLong preferredTimeout) { this.id = requireNonNull(id, "id is null"); this.input = requireNonNull(input, "input is null"); this.operator = requireNonNull(operator, "operator is null"); checkArgument(!nullAllowed || operator == EQUAL, "nullAllowed should be true only with EQUAL operator"); this.nullAllowed = nullAllowed; + this.preferredTimeout = requireNonNull(preferredTimeout, "preferredTimeout is null"); } public Descriptor(DynamicFilterId id, Expression input, Comparison.Operator operator) { - this(id, input, operator, false); + this(id, input, operator, false, OptionalLong.empty()); } public Descriptor(DynamicFilterId id, Expression input) @@ -261,6 +315,11 @@ public boolean isNullAllowed() return nullAllowed; } + public OptionalLong getPreferredTimeout() + { + return preferredTimeout; + } + @Override public boolean equals(Object o) { @@ -274,7 +333,8 @@ public boolean equals(Object o) return Objects.equals(id, that.id) && Objects.equals(input, that.input) && Objects.equals(operator, that.operator) && - nullAllowed == that.nullAllowed; + nullAllowed == that.nullAllowed && + preferredTimeout.equals(that.preferredTimeout); } @Override @@ -291,6 +351,7 @@ public String toString() .add("input", input) .add("operator", operator) .add("nullAllowed", nullAllowed) + .add("timeout", preferredTimeout.isPresent() ? preferredTimeout.getAsLong() : null) .toString(); } @@ -345,28 +406,28 @@ private Function() {} @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") Object input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") Object input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") long input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") long input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") boolean input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") boolean input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") double input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") double input, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @@ -385,28 +446,28 @@ private NullableFunction() {} @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") Object input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") Object input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") long input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") long input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") boolean input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") boolean input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } @TypeParameter("T") @SqlType(BOOLEAN) - public static boolean dynamicFilter(@SqlType("T") double input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed) + public static boolean dynamicFilter(@SqlType("T") double input, @IsNull boolean inputNull, @SqlType(VARCHAR) Slice operator, @SqlType(VARCHAR) Slice id, @SqlType(BOOLEAN) boolean nullAllowed, @SqlNullable @SqlType(INTEGER) Long timeout) { throw new UnsupportedOperationException(); } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/LocalDynamicFiltersCollector.java b/core/trino-main/src/main/java/io/trino/sql/planner/LocalDynamicFiltersCollector.java index 1e31c6009cc8..31ede8718f85 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/LocalDynamicFiltersCollector.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/LocalDynamicFiltersCollector.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -206,5 +207,11 @@ public synchronized TupleDomain getCurrentPredicate() { return currentPredicate; } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } } } diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java index d5528f541fb2..592e9d8a7705 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/PlanOptimizers.java @@ -252,6 +252,7 @@ import io.trino.sql.planner.optimizations.BeginTableWrite; import io.trino.sql.planner.optimizations.CheckSubqueryNodesAreRewritten; import io.trino.sql.planner.optimizations.DeterminePartitionCount; +import io.trino.sql.planner.optimizations.DeterminePreferredDynamicFilterTimeout; import io.trino.sql.planner.optimizations.HashGenerationOptimizer; import io.trino.sql.planner.optimizations.IndexJoinOptimizer; import io.trino.sql.planner.optimizations.LimitPushDown; @@ -964,6 +965,7 @@ public PlanOptimizers( // Remove unsupported dynamic filters introduced by PredicatePushdown. Also, cleanup dynamic filters removed by // PushPredicateIntoTableScan and RemoveRedundantPredicateAboveTableScan due to those rules replacing table scans with empty ValuesNode builder.add(new RemoveUnsupportedDynamicFilters(plannerContext)); + builder.add(new StatsRecordingPlanOptimizer(optimizerStats, new DeterminePreferredDynamicFilterTimeout(statsCalculator))); builder.add(inlineProjections); builder.add(new UnaliasSymbolReferences()); // Run unalias after merging projections to simplify projections more efficiently builder.add(columnPruningOptimizer); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/DeterminePreferredDynamicFilterTimeout.java b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/DeterminePreferredDynamicFilterTimeout.java new file mode 100644 index 000000000000..7691953f4bc5 --- /dev/null +++ b/core/trino-main/src/main/java/io/trino/sql/planner/optimizations/DeterminePreferredDynamicFilterTimeout.java @@ -0,0 +1,321 @@ +/* + * 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.sql.planner.optimizations; + +import com.google.common.collect.ImmutableList; +import io.airlift.units.Duration; +import io.trino.Session; +import io.trino.cost.CachingStatsProvider; +import io.trino.cost.StatsCalculator; +import io.trino.cost.StatsProvider; +import io.trino.cost.SymbolStatsEstimate; +import io.trino.sql.DynamicFilters; +import io.trino.sql.ir.Call; +import io.trino.sql.ir.Expression; +import io.trino.sql.planner.Symbol; +import io.trino.sql.planner.plan.DynamicFilterId; +import io.trino.sql.planner.plan.DynamicFilterSourceNode; +import io.trino.sql.planner.plan.ExchangeNode; +import io.trino.sql.planner.plan.FilterNode; +import io.trino.sql.planner.plan.JoinNode; +import io.trino.sql.planner.plan.PlanNode; +import io.trino.sql.planner.plan.SemiJoinNode; +import io.trino.sql.planner.plan.SimplePlanRewriter; +import io.trino.sql.planner.plan.TableScanNode; +import io.trino.sql.planner.plan.UnionNode; +import io.trino.sql.planner.plan.UnnestNode; +import io.trino.sql.planner.plan.ValuesNode; + +import java.util.AbstractMap.SimpleEntry; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableMap.toImmutableMap; +import static io.trino.SystemSessionProperties.getAwaitedDynamicFilterMaxNdvCount; +import static io.trino.SystemSessionProperties.getAwaitedDynamicFilterMaxRowCount; +import static io.trino.SystemSessionProperties.getPreferredDynamicFilterWaitTimeout; +import static io.trino.sql.DynamicFilters.getDescriptor; +import static io.trino.sql.DynamicFilters.replaceDynamicFilterTimeout; +import static io.trino.sql.ir.IrUtils.combineConjuncts; +import static io.trino.sql.ir.IrUtils.extractConjuncts; +import static io.trino.sql.planner.optimizations.QueryCardinalityUtil.isAtMostScalar; +import static java.lang.Double.isNaN; +import static java.lang.Math.max; +import static java.util.Objects.requireNonNull; + +public class DeterminePreferredDynamicFilterTimeout + implements PlanOptimizer +{ + private final StatsCalculator statsCalculator; + + public DeterminePreferredDynamicFilterTimeout(StatsCalculator statsCalculator) + { + this.statsCalculator = requireNonNull(statsCalculator, "statsCalculator is null"); + } + + @Override + public PlanNode optimize(PlanNode plan, Context context) + { + requireNonNull(plan, "plan is null"); + Session session = context.session(); + + Duration preferredDynamicFilterWaitTimeout = getPreferredDynamicFilterWaitTimeout(session); + long awaitedDynamicFilterMaxRowCount = getAwaitedDynamicFilterMaxRowCount(session); + long awaitedDynamicFilterMaxNdvCount = getAwaitedDynamicFilterMaxNdvCount(session); + + if (preferredDynamicFilterWaitTimeout.toMillis() == 0 || awaitedDynamicFilterMaxRowCount == 0 || awaitedDynamicFilterMaxNdvCount == 0) { + return plan; + } + + Map dynamicFilters = getDynamicFilterSources(plan); + + if (dynamicFilters.isEmpty()) { + return plan; + } + StatsProvider statsProvider = new CachingStatsProvider(statsCalculator, session, context.tableStatsProvider()); + + return SimplePlanRewriter.rewriteWith( + new DeterminePreferredDynamicFilterTimeout.Rewriter( + statsProvider, + preferredDynamicFilterWaitTimeout, + awaitedDynamicFilterMaxRowCount, + awaitedDynamicFilterMaxNdvCount), + plan, + dynamicFilters); + } + + private static Map getDynamicFilterSources(PlanNode plan) + { + return PlanNodeSearcher.searchFrom(plan) + .findAll().stream() + .flatMap(DeterminePreferredDynamicFilterTimeout::getDynamicFiltersMapping) + .collect(toImmutableMap(SimpleEntry::getKey, SimpleEntry::getValue)); + } + + private static Stream> getDynamicFiltersMapping(PlanNode planNode) + { + if (planNode instanceof JoinNode joinNode) { + return joinNode.getDynamicFilters().keySet().stream() + .map(dynamicFilterId -> new SimpleEntry<>(dynamicFilterId, planNode)); + } + if (planNode instanceof DynamicFilterSourceNode dynamicFilterSourceNode) { + return dynamicFilterSourceNode.getDynamicFilters().keySet().stream() + .map(dynamicFilterId -> new SimpleEntry<>(dynamicFilterId, planNode)); + } + if (planNode instanceof SemiJoinNode semiJoinNode && semiJoinNode.getDynamicFilterId().isPresent()) { + return Stream.of(new SimpleEntry<>(semiJoinNode.getDynamicFilterId().get(), planNode)); + } + return Stream.of(); + } + + private static class Rewriter + extends SimplePlanRewriter> + { + private final StatsProvider statsProvider; + private final long preferredDynamicFilterWaitTimeoutMillis; + private final long awaitedDynamicFilterMaxRowCount; + private final long awaitedDynamicFilterMaxNdvCount; + private final Map dynamicFilterBuildSideStates = new HashMap<>(); + + public Rewriter( + StatsProvider statsProvider, + Duration preferredDynamicFilterWaitTimeout, + long awaitedDynamicFilterMaxRowCount, + long awaitedDynamicFilterMaxNdvCount) + { + this.statsProvider = statsProvider; + this.preferredDynamicFilterWaitTimeoutMillis = preferredDynamicFilterWaitTimeout.toMillis(); + this.awaitedDynamicFilterMaxRowCount = awaitedDynamicFilterMaxRowCount; + this.awaitedDynamicFilterMaxNdvCount = awaitedDynamicFilterMaxNdvCount; + } + + @Override + public PlanNode visitFilter(FilterNode node, SimplePlanRewriter.RewriteContext> rewriteContext) + { + if (!(node.getSource() instanceof TableScanNode)) { + // SimplePlanRewriter is visiting all filter nodes, not only ones with dynamic filters. + return visitPlan(node, rewriteContext); + } + + Map dynamicFiltersContext = rewriteContext.get(); + List conjuncts = extractConjuncts(node.getPredicate()); + + ImmutableList.Builder expressionBuilder = ImmutableList.builder(); + for (Expression conjunct : conjuncts) { + Optional descriptor = getDescriptor(conjunct); + if (descriptor.isEmpty()) { + expressionBuilder.add(conjunct); + continue; + } + DynamicFilterId dynamicFilterId = descriptor.get().getId(); + PlanNode planNode = dynamicFiltersContext.get(dynamicFilterId); + + DynamicFilterTimeout dynamicFilterTimeout = dynamicFilterBuildSideStates.computeIfAbsent(dynamicFilterId, ignore -> getBuildSideState(getBuildSide(planNode), getDynamicFilterSymbol(planNode, dynamicFilterId))); + switch (dynamicFilterTimeout) { + case USE_PREFERRED_TIMEOUT -> expressionBuilder.add(replaceDynamicFilterTimeout((Call) conjunct, preferredDynamicFilterWaitTimeoutMillis)); + case NO_WAIT -> expressionBuilder.add(replaceDynamicFilterTimeout((Call) conjunct, 0)); + case UNESTIMATED -> expressionBuilder.add(conjunct); + } + } + + return new FilterNode( + node.getId(), + node.getSource(), + combineConjuncts(expressionBuilder.build())); + } + + private static Symbol getDynamicFilterSymbol(PlanNode planNode, DynamicFilterId dynamicFilterId) + { + if (planNode instanceof JoinNode joinNode) { + return joinNode.getDynamicFilters().get(dynamicFilterId); + } + if (planNode instanceof SemiJoinNode semiJoinNode) { + return semiJoinNode.getFilteringSourceJoinSymbol(); + } + if (planNode instanceof DynamicFilterSourceNode dynamicFilterSourceNode) { + return dynamicFilterSourceNode.getDynamicFilters().get(dynamicFilterId); + } + throw new IllegalArgumentException("Plan node unsupported " + planNode.getClass().getSimpleName()); + } + + private static PlanNode getBuildSide(PlanNode planNode) + { + if (planNode instanceof JoinNode joinNode) { + return joinNode.getRight(); + } + else if (planNode instanceof SemiJoinNode semiJoinNode) { + return semiJoinNode.getFilteringSource(); + } + else if (planNode instanceof DynamicFilterSourceNode dynamicFilterSourceNode) { + return dynamicFilterSourceNode.getSource(); + } + throw new IllegalArgumentException("Plan node unsupported " + planNode.getClass().getSimpleName()); + } + + private DynamicFilterTimeout getBuildSideState(PlanNode planNode, Symbol dynamicFilterSymbol) + { + if (isAtMostScalar(planNode)) { + return DynamicFilterTimeout.USE_PREFERRED_TIMEOUT; + } + + // Skip for expanding plan nodes like CROSS JOIN or UNNEST which can substantially increase the amount of data. + if (isInputMultiplyingPlanNodePresent(planNode)) { + return DynamicFilterTimeout.NO_WAIT; + } + + SymbolStatsEstimate symbolStatsEstimate = statsProvider.getStats(planNode).getSymbolStatistics(dynamicFilterSymbol); + if (!symbolStatsEstimate.isUnknown() && !isExpandingPlanNodePresent(planNode)) { + if (symbolStatsEstimate.getDistinctValuesCount() < awaitedDynamicFilterMaxNdvCount) { + return DynamicFilterTimeout.USE_PREFERRED_TIMEOUT; + } + } + + double rowCount = getEstimatedMaxOutputRowCount(planNode, statsProvider); + if (isNaN(rowCount)) { + return DynamicFilterTimeout.UNESTIMATED; + } + if (rowCount < awaitedDynamicFilterMaxRowCount) { + return DynamicFilterTimeout.USE_PREFERRED_TIMEOUT; + } + + return DynamicFilterTimeout.NO_WAIT; + } + } + + private static boolean isInputMultiplyingPlanNodePresent(PlanNode root) + { + return PlanNodeSearcher.searchFrom(root) + .where(DeterminePreferredDynamicFilterTimeout::isInputMultiplyingPlanNode) + .matches(); + } + + private static boolean isInputMultiplyingPlanNode(PlanNode node) + { + if (node instanceof UnnestNode) { + return true; + } + + if (node instanceof JoinNode joinNode) { + // Skip for cross join + if (joinNode.isCrossJoin()) { + // If any of the input node is scalar then there's no need to skip cross join + return !isAtMostScalar(joinNode.getRight()) && !isAtMostScalar(joinNode.getLeft()); + } + + // Skip for joins with multi keys since output row count stats estimation can wrong due to + // low correlation between multiple join keys. + return joinNode.getCriteria().size() > 1; + } + + return false; + } + + private static boolean isExpandingPlanNodePresent(PlanNode root) + { + return PlanNodeSearcher.searchFrom(root) + .where(DeterminePreferredDynamicFilterTimeout::isExpandingPlanNode) + .matches(); + } + + private static boolean isExpandingPlanNode(PlanNode node) + { + return node instanceof JoinNode + // consider union node and exchange node with multiple sources as expanding since it merge the rows + // from two different sources, thus more data is transferred over the network. + || node instanceof UnionNode + || (node instanceof ExchangeNode && node.getSources().size() > 1); + } + + private static Double getEstimatedMaxOutputRowCount(PlanNode plan, StatsProvider statsProvider) + { + // TODO: this and dependant functions are similar to DeterminePartitionCount and should be extracted to Utils class + double sourceTablesRowCount = getSourceNodesOutputStats(plan, statsProvider); + double expandingNodesMaxRowCount = getExpandingNodesMaxOutputStats(plan, statsProvider); + + return max(sourceTablesRowCount, expandingNodesMaxRowCount); + } + + private static double getSourceNodesOutputStats(PlanNode root, StatsProvider statsProvider) + { + List sourceNodes = PlanNodeSearcher.searchFrom(root) + .whereIsInstanceOfAny(TableScanNode.class, ValuesNode.class) + .findAll(); + + return sourceNodes.stream() + .mapToDouble(node -> statsProvider.getStats(node).getOutputRowCount()) + .sum(); + } + + private static double getExpandingNodesMaxOutputStats(PlanNode root, StatsProvider statsProvider) + { + List expandingNodes = PlanNodeSearcher.searchFrom(root) + .where(DeterminePreferredDynamicFilterTimeout::isExpandingPlanNode) + .findAll(); + + return expandingNodes.stream() + .mapToDouble(node -> statsProvider.getStats(node).getOutputRowCount()) + .max() + .orElse(0); + } + + private enum DynamicFilterTimeout + { + USE_PREFERRED_TIMEOUT, + NO_WAIT, + UNESTIMATED, + } +} 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 359ec00e9a32..a6763a73e3ad 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 @@ -1326,10 +1326,20 @@ private void appendDetailsFromBuilder(NodeRepresentation nodeOutput, StringBuild private String printDynamicFilters(Collection filters) { return filters.stream() - .map(filter -> anonymizer.anonymize(filter.getInput()) + " " + filter.getOperator().getValue() + " #" + filter.getId()) + .map(filter -> anonymizer.anonymize(filter.getInput()) + " " + filter.getOperator().getValue() + " #" + filter.getId() + + printDynamicFilterPreferredTimeout(filter)) .collect(joining(", ", "{", "}")); } + private String printDynamicFilterPreferredTimeout(DynamicFilters.Descriptor filter) + { + if (filter.getPreferredTimeout().isEmpty()) { + return ""; + } + + return format(" await[%d]", MILLISECONDS.toSeconds(filter.getPreferredTimeout().getAsLong())); + } + private String printDynamicFilterAssignments(Map filters) { return filters.entrySet().stream() diff --git a/core/trino-main/src/test/java/io/trino/execution/TestDynamicFilterConfig.java b/core/trino-main/src/test/java/io/trino/execution/TestDynamicFilterConfig.java index ac5950ffcf05..80e889996db8 100644 --- a/core/trino-main/src/test/java/io/trino/execution/TestDynamicFilterConfig.java +++ b/core/trino-main/src/test/java/io/trino/execution/TestDynamicFilterConfig.java @@ -15,9 +15,11 @@ import com.google.common.collect.ImmutableMap; import io.airlift.units.DataSize; +import io.airlift.units.Duration; import org.junit.jupiter.api.Test; import java.util.Map; +import java.util.concurrent.TimeUnit; import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; @@ -44,6 +46,9 @@ public void testDefaults() .setSmallPartitionedRangeRowLimitPerDriver(500) .setSmallPartitionedMaxSizePerOperator(DataSize.of(500, KILOBYTE)) .setSmallMaxSizePerFilter(DataSize.of(5, MEGABYTE)) + .setPreferredDynamicFilterWaitTimeout(new Duration(20, TimeUnit.SECONDS)) + .setAwaitedDynamicFilterMaxRowCount(100_000) + .setAwaitedDynamicFilterMaxNdvCount(500) .setLargeMaxDistinctValuesPerDriver(10_000) .setLargeMaxSizePerDriver(DataSize.of(2, MEGABYTE)) .setLargeRangeRowLimitPerDriver(20_000) @@ -63,6 +68,9 @@ public void testExplicitPropertyMappings() .put("enable-large-dynamic-filters", "true") .put("enable-dynamic-row-filtering", "false") .put("dynamic-row-filtering.selectivity-threshold", "0.8") + .put("preferred-dynamic-filter.wait-timeout", "50s") + .put("awaited-dynamic-filter.max-row-count", "500000") + .put("awaited-dynamic-filter.max-ndv-count", "2000") .put("dynamic-filtering.small.max-distinct-values-per-driver", "256") .put("dynamic-filtering.small.max-size-per-driver", "64kB") .put("dynamic-filtering.small.range-row-limit-per-driver", "20000") @@ -88,6 +96,9 @@ public void testExplicitPropertyMappings() .setEnableLargeDynamicFilters(true) .setEnableDynamicRowFiltering(false) .setDynamicRowFilterSelectivityThreshold(0.8) + .setAwaitedDynamicFilterMaxRowCount(500_000) + .setAwaitedDynamicFilterMaxNdvCount(2000) + .setPreferredDynamicFilterWaitTimeout(new Duration(50, TimeUnit.SECONDS)) .setSmallMaxDistinctValuesPerDriver(256) .setSmallMaxSizePerDriver(DataSize.of(64, KILOBYTE)) .setSmallRangeRowLimitPerDriver(20000) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFiltersCollector.java b/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFiltersCollector.java index a3d424ba8d00..76905a51b124 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFiltersCollector.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/TestLocalDynamicFiltersCollector.java @@ -31,6 +31,7 @@ import java.util.List; import java.util.Map; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -248,8 +249,8 @@ public void testIsNotDistinctFrom() DynamicFilter filter = createDynamicFilter( collector, ImmutableList.of( - new DynamicFilters.Descriptor(filterId1, symbol1.toSymbolReference(), EQUAL, true), - new DynamicFilters.Descriptor(filterId2, symbol2.toSymbolReference(), EQUAL, true)), + new DynamicFilters.Descriptor(filterId1, symbol1.toSymbolReference(), EQUAL, true, OptionalLong.empty()), + new DynamicFilters.Descriptor(filterId2, symbol2.toSymbolReference(), EQUAL, true, OptionalLong.empty())), ImmutableMap.of(symbol1, column1, symbol2, column2)); assertThat(filter.getColumnsCovered()) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/JoinMatcher.java b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/JoinMatcher.java index 93b774757ca8..40e32a020709 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/JoinMatcher.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/JoinMatcher.java @@ -32,10 +32,12 @@ import io.trino.sql.planner.plan.JoinType; import io.trino.sql.planner.plan.PlanNode; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import static com.google.common.base.MoreObjects.toStringHelper; @@ -173,7 +175,9 @@ private boolean matchDynamicFilters(JoinNode joinNode, SymbolAliases symbolAlias .filter(descriptor -> dynamicFilterIds.contains(descriptor.getId())) .collect(toImmutableList()); - Set actual = new HashSet<>(); + Set actualExpressions = new HashSet<>(); + Map actualTimeouts = new HashMap<>(); + for (DynamicFilters.Descriptor descriptor : descriptors) { Expression probe = descriptor.getInput(); Symbol build = idToBuildSymbolMap.get(descriptor.getId()); @@ -187,14 +191,27 @@ private boolean matchDynamicFilters(JoinNode joinNode, SymbolAliases symbolAlias else { expression = new Comparison(descriptor.getOperator(), probe, build.toSymbolReference()); } - actual.add(expression); + actualExpressions.add(expression); + if (descriptor.getPreferredTimeout().isPresent()) { + actualTimeouts.put(expression, descriptor.getPreferredTimeout()); + } } - Set expected = expectedDynamicFilter.get().stream() - .map(pattern -> pattern.getExpression(symbolAliases)) + Map expectedTimeouts = new HashMap<>(); + Set expectedExpressions = expectedDynamicFilter.get().stream() + .map(pattern -> { + Expression expression = pattern.getExpression(symbolAliases); + if (pattern.getPreferredTimeout().isPresent()) { + expectedTimeouts.put(expression, pattern.getPreferredTimeout()); + } + return expression; + }) .collect(toImmutableSet()); - return expected.equals(actual); + boolean timeoutsMatch = expectedTimeouts.entrySet().stream() + .allMatch(expectedTimeout -> expectedTimeout.getValue().equals(actualTimeouts.get(expectedTimeout.getKey()))); + + return expectedExpressions.equals(actualExpressions) && timeoutsMatch; } @Override @@ -270,6 +287,14 @@ public Builder dynamicFilter(Type type, String key, String value) return this; } + @CanIgnoreReturnValue + public Builder dynamicFilter(Reference symbolKey, String value, long timeout) + { + this.dynamicFilter = Optional.of(ImmutableList.of(new PlanMatchPattern.DynamicFilterPattern(symbolKey, EQUAL, value, false, OptionalLong.of(timeout)))); + + return this; + } + @CanIgnoreReturnValue public Builder dynamicFilter(List expectedDynamicFilter) { diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java index bbe26f05e3d0..686fb4f836be 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/assertions/PlanMatchPattern.java @@ -78,6 +78,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.function.Consumer; import java.util.function.Predicate; @@ -1228,13 +1229,25 @@ public static class DynamicFilterPattern private final Comparison.Operator operator; private final SymbolAlias build; private final boolean nullAllowed; + private final OptionalLong preferredTimeout; public DynamicFilterPattern(Expression probe, Comparison.Operator operator, String buildAlias, boolean nullAllowed) + { + this( + probe, + operator, + buildAlias, + nullAllowed, + OptionalLong.empty()); + } + + public DynamicFilterPattern(Expression probe, Comparison.Operator operator, String buildAlias, boolean nullAllowed, OptionalLong preferredTimeout) { this.probe = requireNonNull(probe, "probe is null"); this.operator = requireNonNull(operator, "operator is null"); this.build = new SymbolAlias(requireNonNull(buildAlias, "buildAlias is null")); this.nullAllowed = nullAllowed; + this.preferredTimeout = requireNonNull(preferredTimeout, "minDynamicFilterTimeout is null"); } public DynamicFilterPattern(Expression probe, Comparison.Operator operator, String buildAlias) @@ -1257,6 +1270,11 @@ Expression getExpression(SymbolAliases aliases) build.toSymbol(aliases).toSymbolReference()); } + public OptionalLong getPreferredTimeout() + { + return preferredTimeout; + } + private static SymbolMapper symbolMapper(SymbolAliases symbolAliases) { return new SymbolMapper(symbol -> Symbol.from(symbolAliases.get(symbol.name()))); @@ -1269,6 +1287,7 @@ public String toString() .add("probe", probe) .add("operator", operator) .add("build", build) + .add("preferredTimeout", preferredTimeout) .toString(); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestDeterminePreferredDynamicFilterTimeout.java b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestDeterminePreferredDynamicFilterTimeout.java new file mode 100644 index 000000000000..f1f4c1f26365 --- /dev/null +++ b/core/trino-main/src/test/java/io/trino/sql/planner/optimizations/TestDeterminePreferredDynamicFilterTimeout.java @@ -0,0 +1,466 @@ +/* + * 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.sql.planner.optimizations; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.connector.MockConnectorColumnHandle; +import io.trino.connector.MockConnectorFactory; +import io.trino.connector.MockConnectorTableHandle; +import io.trino.metadata.ResolvedFunction; +import io.trino.metadata.TestingFunctionResolution; +import io.trino.spi.connector.ColumnMetadata; +import io.trino.spi.statistics.ColumnStatistics; +import io.trino.spi.statistics.Estimate; +import io.trino.spi.statistics.TableStatistics; +import io.trino.sql.ir.Between; +import io.trino.sql.ir.Call; +import io.trino.sql.ir.Comparison; +import io.trino.sql.ir.Constant; +import io.trino.sql.ir.Reference; +import io.trino.sql.planner.assertions.BasePlanTest; +import io.trino.sql.planner.assertions.PlanMatchPattern.DynamicFilterPattern; +import io.trino.sql.planner.plan.ExchangeNode; +import io.trino.sql.planner.plan.FilterNode; +import io.trino.testing.PlanTester; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalLong; +import java.util.Set; + +import static io.trino.SystemSessionProperties.AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT; +import static io.trino.SystemSessionProperties.AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT; +import static io.trino.SystemSessionProperties.ENABLE_DYNAMIC_FILTERING; +import static io.trino.SystemSessionProperties.ENABLE_STATS_CALCULATOR; +import static io.trino.SystemSessionProperties.FILTERING_SEMI_JOIN_TO_INNER; +import static io.trino.SystemSessionProperties.JOIN_DISTRIBUTION_TYPE; +import static io.trino.SystemSessionProperties.JOIN_REORDERING_STRATEGY; +import static io.trino.SystemSessionProperties.getPreferredDynamicFilterWaitTimeout; +import static io.trino.spi.statistics.TableStatistics.empty; +import static io.trino.spi.type.BigintType.BIGINT; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.sql.DynamicFilters.extractDynamicFilters; +import static io.trino.sql.analyzer.TypeSignatureProvider.fromTypes; +import static io.trino.sql.ir.Booleans.TRUE; +import static io.trino.sql.ir.Comparison.Operator.EQUAL; +import static io.trino.sql.ir.Comparison.Operator.GREATER_THAN; +import static io.trino.sql.ir.Comparison.Operator.GREATER_THAN_OR_EQUAL; +import static io.trino.sql.ir.Comparison.Operator.LESS_THAN_OR_EQUAL; +import static io.trino.sql.planner.OptimizerConfig.JoinDistributionType.BROADCAST; +import static io.trino.sql.planner.OptimizerConfig.JoinReorderingStrategy.NONE; +import static io.trino.sql.planner.assertions.PlanMatchPattern.aggregation; +import static io.trino.sql.planner.assertions.PlanMatchPattern.aggregationFunction; +import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; +import static io.trino.sql.planner.assertions.PlanMatchPattern.equiJoinClause; +import static io.trino.sql.planner.assertions.PlanMatchPattern.exchange; +import static io.trino.sql.planner.assertions.PlanMatchPattern.filter; +import static io.trino.sql.planner.assertions.PlanMatchPattern.join; +import static io.trino.sql.planner.assertions.PlanMatchPattern.node; +import static io.trino.sql.planner.assertions.PlanMatchPattern.semiJoin; +import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; +import static io.trino.sql.planner.plan.AggregationNode.Step.FINAL; +import static io.trino.sql.planner.plan.AggregationNode.Step.PARTIAL; +import static io.trino.sql.planner.plan.ExchangeNode.Scope.LOCAL; +import static io.trino.sql.planner.plan.JoinType.INNER; +import static io.trino.testing.TestingSession.testSessionBuilder; + +public class TestDeterminePreferredDynamicFilterTimeout + extends BasePlanTest +{ + private static final ResolvedFunction RANDOM = new TestingFunctionResolution().resolveFunction("random", fromTypes(INTEGER)); + + private long waitForCascadingDynamicFiltersTimeout; + + @Override + protected PlanTester createPlanTester() + { + String catalogName = "mock"; + Map sessionProperties = ImmutableMap.of( + ENABLE_DYNAMIC_FILTERING, "true", + JOIN_REORDERING_STRATEGY, NONE.name(), + JOIN_DISTRIBUTION_TYPE, BROADCAST.name()); + Map tables = Map.of( + "table_undefined_a", new TableStatistics( + Estimate.unknown(), + Map.of( + new MockConnectorColumnHandle("a_1", INTEGER), ColumnStatistics.empty(), + new MockConnectorColumnHandle("a_2", INTEGER), ColumnStatistics.empty())), + "table_undefined_b", new TableStatistics( + Estimate.unknown(), + Map.of( + new MockConnectorColumnHandle("b_1", INTEGER), ColumnStatistics.empty(), + new MockConnectorColumnHandle("b_2", INTEGER), ColumnStatistics.empty())), + "table_small_a", new TableStatistics( + Estimate.of(10_000), + Map.of( + new MockConnectorColumnHandle("a_1", INTEGER), ColumnStatistics.empty(), + new MockConnectorColumnHandle("a_2", INTEGER), new ColumnStatistics( + Estimate.unknown(), Estimate.of(400), Estimate.unknown(), Optional.empty()))), + "table_small_b", new TableStatistics( + Estimate.of(10_000), + Map.of( + new MockConnectorColumnHandle("b_1", INTEGER), ColumnStatistics.empty(), + new MockConnectorColumnHandle("b_2", INTEGER), new ColumnStatistics( + Estimate.unknown(), Estimate.of(400), Estimate.unknown(), Optional.empty()), + new MockConnectorColumnHandle("b_3", INTEGER), ColumnStatistics.empty())), + "table_small_c", new TableStatistics( + Estimate.of(10_000), + Map.of( + new MockConnectorColumnHandle("c_1", INTEGER), ColumnStatistics.empty(), + new MockConnectorColumnHandle("c_2", INTEGER), new ColumnStatistics( + Estimate.unknown(), Estimate.of(400), Estimate.unknown(), Optional.empty())))); + MockConnectorFactory connectorFactory = MockConnectorFactory.builder() + .withGetTableHandle((session, table) -> { + if (tables.containsKey(table.getTableName())) { + return new MockConnectorTableHandle(table); + } + return null; + }) + .withGetColumns(schemaTableName -> { + TableStatistics table = tables.get(schemaTableName.getTableName()); + if (table != null) { + return table.getColumnStatistics().keySet().stream() + .map(MockConnectorColumnHandle.class::cast) + .map(columnHandle -> new ColumnMetadata(columnHandle.getName(), columnHandle.getType())) + .toList(); + } + return Collections.emptyList(); + }) + .withGetTableStatistics(tableName -> { + TableStatistics table = tables.get(tableName.getTableName()); + if (table != null) { + return table; + } + else { + return empty(); + } + }) + .withName(catalogName) + .build(); + + Session.SessionBuilder sessionBuilder = testSessionBuilder() + .setCatalog(catalogName) + .setSchema("default"); + sessionProperties.forEach(sessionBuilder::setSystemProperty); + + PlanTester planTester = PlanTester.create(sessionBuilder.build()); + planTester.createCatalog( + catalogName, + connectorFactory, + ImmutableMap.of()); + return planTester; + } + + @BeforeAll + public void setup() + { + waitForCascadingDynamicFiltersTimeout = getPreferredDynamicFilterWaitTimeout(getPlanTester().getDefaultSession()).toMillis(); + } + + @Test + public void testWithUnknownTableSize() + { + assertPlan( + "SELECT table_undefined_a.a_1 from table_undefined_a, table_undefined_b where table_undefined_a.a_1 = table_undefined_b.b_1", + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_1") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_1", false, OptionalLong.empty()))) + .left( + node(FilterNode.class, + tableScan("table_undefined_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_undefined_b", ImmutableMap.of("B_1", "b_1"))))))); + } + + @Test + public void testWithSmallRowCountTable() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_1 = table_small_b.b_1", + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_1") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_1", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_1", "b_1"))))))); + } + + @Test + public void testWithExceedingRowCount() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_1 = table_small_b.b_1", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT, "9999") + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_1") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_1", false, OptionalLong.of(0L)))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_1", "b_1"))))))); + } + + @Test + public void testWithSmallNdvCount() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_1 = table_small_b.b_2", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT, "1") + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT, "500") + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_2") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_2", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_2", "b_2"))))))); + } + + @Test + public void testWithExceedNdvCount() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_1 = table_small_b.b_2", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_ROW_COUNT, "9999") + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT, "399") + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_2") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_2", false, OptionalLong.of(0L)))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_2", "b_2"))))))); + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_2 = table_small_b.b_2", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(AWAITED_DYNAMIC_FILTER_MAX_NDV_COUNT, "399") + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_2", "B_2") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_2"), EQUAL, "B_2", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_2", "a_2")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_2", "b_2"))))))); + } + + @Test + public void testWithDisabledStatistics() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a, table_small_b where table_small_a.a_1 = table_small_b.b_1", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(ENABLE_STATS_CALCULATOR, "false") + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_1") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), EQUAL, "B_1", false, OptionalLong.empty()))) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + tableScan("table_small_b", ImmutableMap.of("B_1", "b_1"))))))); + } + + @Test + public void testDependantDynamicFilterTable() + { + assertPlan( + "SELECT table_small_c.c_1 from (table_small_a JOIN table_small_b ON table_small_a.a_2 = table_small_b.b_2) " + + "JOIN table_small_c ON table_small_a.a_2 = table_small_c.c_1", + Session.builder(getPlanTester().getDefaultSession()) + .build(), + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_2", "C_1") + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_2"), EQUAL, "C_1", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)), + new DynamicFilterPattern(new Reference(INTEGER, "B_2"), EQUAL, "C_1", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + join(INNER, leftJoinBuilder -> leftJoinBuilder + .equiCriteria("A_2", "B_2") + .dynamicFilter(new Reference(INTEGER, "A_2"), "B_2", waitForCascadingDynamicFiltersTimeout) + .left( + node(FilterNode.class, + tableScan("table_small_a", ImmutableMap.of("A_2", "a_2")))) + .right( + anyTree(node(FilterNode.class, + tableScan("table_small_b", ImmutableMap.of("B_2", "b_2"))))))) + .right( + exchange( + tableScan("table_small_c", ImmutableMap.of("C_1", "c_1"))))))); + } + + @Test + public void testWithExpandingNodeAndSmallRowCount() + { + assertPlan( + "SELECT a.a_1 FROM table_small_a a JOIN (SELECT b_1 FROM table_small_b UNION ALL SELECT b_1 FROM table_small_b) b ON a.a_1 = b.b_1", + anyTree( + join(INNER, builder -> builder + .equiCriteria("A_1", "B_1") + .dynamicFilter(new Reference(BIGINT, "A_1"), "B_1", waitForCascadingDynamicFiltersTimeout) + .left( + anyTree( + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1")))) + .right( + exchange( + LOCAL, + Optional.empty(), + Optional.empty(), + ImmutableList.of(), + Set.of(), + Optional.empty(), + ImmutableList.of("B_1"), + Optional.empty(), + tableScan("table_small_b", ImmutableMap.of("B_1_1", "b_1")), + tableScan("table_small_b", ImmutableMap.of("B_1_2", "b_1"))))))); + } + + @Test + public void testWithMultiplyingNode() + { + assertPlan(""" + SELECT a.a_1, a.a_2 FROM table_small_a a,table_small_b b + JOIN table_small_c c ON b.b_2 = c.c_1 AND c.c_2 = b.b_3 + WHERE a.a_1 BETWEEN b.b_1 AND b.b_2 + """, + anyTree(filter( + new Between(new Reference(INTEGER, "A_1"), new Reference(INTEGER, "B_1"), new Reference(INTEGER, "B_2")), + join(INNER, builder -> builder + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), GREATER_THAN_OR_EQUAL, "B_1", false, OptionalLong.of(0L)), + new DynamicFilterPattern(new Reference(INTEGER, "A_1"), LESS_THAN_OR_EQUAL, "B_2", false, OptionalLong.of(0L)))) + .left( + filter( + TRUE, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1", "A_2", "a_2")))) + .right( + exchange( + LOCAL, + join(INNER, innerJoinBuilder -> innerJoinBuilder + .equiCriteria(ImmutableList.of( + equiJoinClause("B_2", "C_1"), + equiJoinClause("B_3", "C_2"))) + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(INTEGER, "B_2"), EQUAL, "C_1", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)), + new DynamicFilterPattern(new Reference(INTEGER, "B_3"), EQUAL, "C_2", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + anyTree( + tableScan("table_small_b", ImmutableMap.of("B_1", "b_1", "B_2", "b_2", "B_3", "b_3")))) + .right( + exchange( + LOCAL, + tableScan("table_small_c", ImmutableMap.of("C_2", "c_2", "C_1", "c_1"))))))))))); + } + + @Test + public void testWithSemiJoinAndSmallRowCount() + { + assertPlan( + "SELECT table_small_a.a_1 from table_small_a where table_small_a.a_1 IN (SELECT b_1 from table_small_b where b_1 = random(5))", + Session.builder(getPlanTester().getDefaultSession()) + .setSystemProperty(FILTERING_SEMI_JOIN_TO_INNER, "false") + .build(), + anyTree( + semiJoin("A_1", "B_1", "SEMI_JOIN_RESULT", true, + filter(TRUE, + tableScan("table_small_a", ImmutableMap.of("A_1", "a_1"))) + .with(FilterNode.class, filterNode -> extractDynamicFilters(filterNode.getPredicate()) + .getDynamicConjuncts().get(0).getPreferredTimeout() + .equals(OptionalLong.of(waitForCascadingDynamicFiltersTimeout))), + node(ExchangeNode.class, + filter( + new Comparison(EQUAL, new Reference(INTEGER, "B_1"), new Call(RANDOM, ImmutableList.of(new Constant(INTEGER, 5L)))), + tableScan("table_small_b", ImmutableMap.of("B_1", "b_1"))))))); + } + + @Test + public void testWithScalar() + { + assertPlan( + """ + SELECT a.a_1 FROM table_undefined_a a + INNER JOIN (SELECT max(b_1) AS max_b FROM table_undefined_b) AS b + ON a.a_1 > b.max_b + """, + anyTree( + anyTree( + filter( + new Comparison(GREATER_THAN, new Reference(BIGINT, "A_1"), new Reference(BIGINT, "MAX_1")), + join(INNER, builder -> builder + .dynamicFilter( + ImmutableList.of( + new DynamicFilterPattern(new Reference(BIGINT, "A_1"), GREATER_THAN, "MAX_1", false, OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .left( + filter(TRUE, + tableScan("table_undefined_a", ImmutableMap.of("A_1", "a_1"))) + .with(FilterNode.class, filterNode -> extractDynamicFilters(filterNode.getPredicate()) + .getDynamicConjuncts().get(0).getPreferredTimeout() + .equals(OptionalLong.of(waitForCascadingDynamicFiltersTimeout)))) + .right( + aggregation(ImmutableMap.of("MAX_1", aggregationFunction("max", ImmutableList.of("MAX_2"))), FINAL, + anyTree( + aggregation(ImmutableMap.of("MAX_2", aggregationFunction("max", ImmutableList.of("B_1"))), PARTIAL, + tableScan("table_undefined_b", ImmutableMap.of("B_1", "b_1"))))))))))); + } +} diff --git a/core/trino-main/src/test/java/io/trino/util/DynamicFiltersTestUtil.java b/core/trino-main/src/test/java/io/trino/util/DynamicFiltersTestUtil.java index 5ed1ce1767e0..59180504d561 100644 --- a/core/trino-main/src/test/java/io/trino/util/DynamicFiltersTestUtil.java +++ b/core/trino-main/src/test/java/io/trino/util/DynamicFiltersTestUtil.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Objects; +import java.util.OptionalLong; import java.util.Set; import java.util.StringJoiner; import java.util.concurrent.CompletableFuture; @@ -164,6 +165,12 @@ public TupleDomain getCurrentPredicate() return currentPredicate; } + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.empty(); + } + @Override public boolean equals(Object o) { diff --git a/core/trino-spi/pom.xml b/core/trino-spi/pom.xml index 113f5a5b04c8..21a5125727d5 100644 --- a/core/trino-spi/pom.xml +++ b/core/trino-spi/pom.xml @@ -285,6 +285,12 @@ java.method.removed method long io.trino.spi.block.RunLengthEncodedBlock::getLogicalSizeInBytes() + + true + java.method.addedToInterface + method java.util.OptionalLong io.trino.spi.connector.DynamicFilter::getPreferredDynamicFilterTimeout() + Internal change + diff --git a/core/trino-spi/src/main/java/io/trino/spi/connector/DynamicFilter.java b/core/trino-spi/src/main/java/io/trino/spi/connector/DynamicFilter.java index 50cacc6e68c0..073d26ddf76d 100644 --- a/core/trino-spi/src/main/java/io/trino/spi/connector/DynamicFilter.java +++ b/core/trino-spi/src/main/java/io/trino/spi/connector/DynamicFilter.java @@ -15,6 +15,7 @@ import io.trino.spi.predicate.TupleDomain; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -53,6 +54,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); // no filtering } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.empty(); + } }; /** @@ -81,4 +88,12 @@ public TupleDomain getCurrentPredicate() boolean isAwaitable(); TupleDomain getCurrentPredicate(); + + /** + * Returns preferred timeout in milliseconds if build side can be estimated otherwise empty, that connector should wait + * for the dynamic filter to be narrowed down since split enumeration started. + * Future from {@link DynamicFilter#isBlocked()} method should be acquired before getting preferred dynamic filter timeout. + * This timeout needs to be re-checked whenever connector decides to wait for dynamic filter. + */ + OptionalLong getPreferredDynamicFilterTimeout(); } diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringConfig.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringConfig.java index 2791158eee7d..b6faad0cc2c8 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringConfig.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringConfig.java @@ -23,8 +23,6 @@ public class JdbcDynamicFilteringConfig { private boolean dynamicFilteringEnabled = true; - // 20s allows DF collection from dimensional tables as well as - // some larger build side subqueries private Duration dynamicFilteringWaitTimeout = new Duration(20, SECONDS); public boolean isDynamicFilteringEnabled() diff --git a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringSplitManager.java b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringSplitManager.java index 0b383d80d91f..e813e5f60371 100644 --- a/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringSplitManager.java +++ b/plugin/trino-base-jdbc/src/main/java/io/trino/plugin/jdbc/JdbcDynamicFilteringSplitManager.java @@ -27,14 +27,18 @@ import io.trino.spi.connector.DynamicFilter; import java.util.Optional; +import java.util.OptionalLong; import java.util.concurrent.CompletableFuture; import static io.airlift.units.Duration.succinctNanos; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.dynamicFilteringEnabled; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.getDynamicFilteringWaitTimeout; import static io.trino.spi.connector.ConnectorSplitSource.ConnectorSplitBatch; +import static java.lang.Math.max; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; /** * Implements waiting for collection of dynamic filters before generating splits from {@link ConnectorSplitManager}. @@ -47,6 +51,7 @@ public class JdbcDynamicFilteringSplitManager { private static final Logger log = Logger.get(JdbcDynamicFilteringSplitManager.class); private static final ConnectorSplitBatch EMPTY_BATCH = new ConnectorSplitBatch(ImmutableList.of(), false); + private static final long MIN_PREFERRED_DYNAMIC_FILTER_WAIT_TIMEOUT_NANOS = SECONDS.toNanos(1); private final ConnectorSplitManager delegateSplitManager; private final DynamicFilteringStats stats; @@ -116,7 +121,8 @@ private class DynamicFilteringSplitSource @Override public CompletableFuture getNextBatch(int maxSize) { - long remainingTimeoutNanos = getRemainingTimeoutNanos(); + CompletableFuture blocked = dynamicFilter.isBlocked(); + long remainingTimeoutNanos = getRemainingTimeoutNanos(dynamicFilter); if (remainingTimeoutNanos > 0 && dynamicFilter.isAwaitable()) { log.debug( "Waiting for dynamic filter (query: %s, table: %s, remaining timeout: %s)", @@ -124,7 +130,7 @@ public CompletableFuture getNextBatch(int maxSize) table, succinctNanos(remainingTimeoutNanos)); // wait for dynamic filter and yield - return dynamicFilter.isBlocked() + return blocked .thenApply(_ -> EMPTY_BATCH) .completeOnTimeout(EMPTY_BATCH, remainingTimeoutNanos, NANOSECONDS); } @@ -149,16 +155,21 @@ public void close() @Override public boolean isFinished() { - if (getRemainingTimeoutNanos() > 0 && dynamicFilter.isAwaitable()) { + if (getRemainingTimeoutNanos(dynamicFilter) > 0 && dynamicFilter.isAwaitable()) { return false; } return getDelegateSplitSource().isFinished(); } - private long getRemainingTimeoutNanos() + private long getRemainingTimeoutNanos(DynamicFilter dynamicFilter) { - return dynamicFilteringTimeoutNanos - (System.nanoTime() - startNanos); + long currentDynamicFilterTimeoutNanos = dynamicFilteringTimeoutNanos; + OptionalLong preferredDynamicFilterTimeout = dynamicFilter.getPreferredDynamicFilterTimeout(); + if (preferredDynamicFilterTimeout.isPresent()) { + currentDynamicFilterTimeoutNanos = max(MIN_PREFERRED_DYNAMIC_FILTER_WAIT_TIMEOUT_NANOS, MILLISECONDS.toNanos(preferredDynamicFilterTimeout.getAsLong())); + } + return currentDynamicFilterTimeoutNanos - (System.nanoTime() - startNanos); } private synchronized ConnectorSplitSource getDelegateSplitSource() diff --git a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDynamicFilteringSplitManager.java b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDynamicFilteringSplitManager.java index 68b43110f8d3..2aff112b0512 100644 --- a/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDynamicFilteringSplitManager.java +++ b/plugin/trino-base-jdbc/src/test/java/io/trino/plugin/jdbc/TestJdbcDynamicFilteringSplitManager.java @@ -28,10 +28,13 @@ import io.trino.testing.TestingTransactionHandle; import org.junit.jupiter.api.Test; +import java.util.List; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; +import static com.google.common.collect.ImmutableList.toImmutableList; import static io.airlift.concurrent.MoreFutures.unmodifiableFuture; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.DYNAMIC_FILTERING_ENABLED; import static io.trino.plugin.jdbc.JdbcDynamicFilteringSessionProperties.DYNAMIC_FILTERING_WAIT_TIMEOUT; @@ -42,12 +45,6 @@ public class TestJdbcDynamicFilteringSplitManager { private static final ConnectorTransactionHandle TRANSACTION_HANDLE = TestingTransactionHandle.create(); - private static final ConnectorSession SESSION = TestingConnectorSession.builder() - .setPropertyMetadata(new JdbcDynamicFilteringSessionProperties(new JdbcDynamicFilteringConfig()).getSessionProperties()) - .setPropertyValues(ImmutableMap.of( - DYNAMIC_FILTERING_WAIT_TIMEOUT, "3s", - DYNAMIC_FILTERING_ENABLED, true)) - .build(); private static final JdbcTableHandle TABLE_HANDLE = new JdbcTableHandle( new SchemaTableName("schema", "table"), new RemoteTableName(Optional.empty(), Optional.empty(), "table"), @@ -83,22 +80,25 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } }; @Test public void testBlockingTimeout() throws Exception { - JdbcDynamicFilteringSplitManager manager = new JdbcDynamicFilteringSplitManager( - new TestingSplitManager(ImmutableList.of()), - new DynamicFilteringStats()); - ConnectorSplitSource splitSource = manager.getSplits( - TRANSACTION_HANDLE, - SESSION, - TABLE_HANDLE, - BLOCKED_DYNAMIC_FILTER, - alwaysTrue()); - + TestingConnectorSession session = TestingConnectorSession.builder() + .setPropertyMetadata(new JdbcDynamicFilteringSessionProperties(new JdbcDynamicFilteringConfig()).getSessionProperties()) + .setPropertyValues(ImmutableMap.of( + DYNAMIC_FILTERING_WAIT_TIMEOUT, "3s", + DYNAMIC_FILTERING_ENABLED, true)) + .build(); + ConnectorSplitSource splitSource = getConnectorSplitSource(session, BLOCKED_DYNAMIC_FILTER); // verify that getNextBatch() future completes after a timeout CompletableFuture future = splitSource.getNextBatch(100); assertThat(future.isDone()).isFalse(); @@ -106,4 +106,101 @@ public void testBlockingTimeout() assertThat(splitSource.isFinished()).isTrue(); splitSource.close(); } + + @Test + public void testMinDynamicFilterBlockingTimeout() + throws Exception + { + TestingConnectorSession session = TestingConnectorSession.builder() + .setPropertyMetadata(new JdbcDynamicFilteringSessionProperties(new JdbcDynamicFilteringConfig()).getSessionProperties()) + .setPropertyValues(ImmutableMap.of( + DYNAMIC_FILTERING_WAIT_TIMEOUT, "0s", + DYNAMIC_FILTERING_ENABLED, true)) + .build(); + CompletableFuture dynamicFilterFuture = new CompletableFuture<>(); + DynamicFilter dynamicFilter = new DynamicFilter() + { + final List> lazyDynamicFilterFutures = ImmutableList.of( + dynamicFilterFuture, + new CompletableFuture<>()); + + @Override + public Set getColumnsCovered() + { + return ImmutableSet.of(); + } + + @Override + public CompletableFuture isBlocked() + { + return unmodifiableFuture(CompletableFuture.anyOf(getUndoneFutures().toArray(new CompletableFuture[0]))); + } + + @Override + public boolean isComplete() + { + return getUndoneFutures().isEmpty(); + } + + @Override + public boolean isAwaitable() + { + return !isComplete(); + } + + @Override + public TupleDomain getCurrentPredicate() + { + return TupleDomain.all(); + } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return getUndoneFutures().isEmpty() ? OptionalLong.of(0L) : OptionalLong.of(3000L); + } + + private List> getUndoneFutures() + { + return lazyDynamicFilterFutures.stream() + .filter(future -> !future.isDone()) + .collect(toImmutableList()); + } + }; + ConnectorSplitSource splitSource = getConnectorSplitSource(session, dynamicFilter); + + // verify that getNextBatch() future completes after a min dynamic filter timeout + CompletableFuture splitSourceNextBatchFuture = splitSource.getNextBatch(100); + assertThat(splitSourceNextBatchFuture.isDone()).isFalse(); + // first narrow down of DF + dynamicFilterFuture.complete(null); + assertThat(splitSourceNextBatchFuture.isDone()).isTrue(); + // whole DF is not completed, still min dynamic filter timeout remains + assertThat(splitSource.isFinished()).isFalse(); + splitSourceNextBatchFuture = splitSource.getNextBatch(100); + assertThat(splitSourceNextBatchFuture.isDone()).isFalse(); + assertThat(splitSource.isFinished()).isFalse(); + // await preferred timeout ~ 3s + splitSourceNextBatchFuture.get(20, SECONDS); + assertThat(splitSourceNextBatchFuture.isDone()).isTrue(); + // preferred timeout passed but dynamic filter is still not done + assertThat(dynamicFilter.isAwaitable()).isTrue(); + // split source is completed + assertThat(splitSource.isFinished()).isTrue(); + splitSource.close(); + } + + private static ConnectorSplitSource getConnectorSplitSource(ConnectorSession session, DynamicFilter blockedDynamicFilter) + { + JdbcDynamicFilteringSplitManager manager = new JdbcDynamicFilteringSplitManager( + new TestingSplitManager(ImmutableList.of()), + new DynamicFilteringStats()); + + return manager.getSplits( + TRANSACTION_HANDLE, + session, + TABLE_HANDLE, + blockedDynamicFilter, + alwaysTrue()); + } } 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 6f1bc9c9d473..8dfb6ff8f9fe 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 @@ -50,6 +50,7 @@ import static io.airlift.concurrent.MoreFutures.toCompletableFuture; import static io.trino.plugin.deltalake.DeltaLakeSplitManager.partitionMatchesPredicate; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; +import static java.lang.Math.max; import static java.util.Objects.requireNonNull; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -118,9 +119,10 @@ private ErrorCodeSupplier findErrorCode(Throwable throwable) @Override public CompletableFuture getNextBatch(int maxSize) { - long timeLeft = dynamicFilteringWaitTimeoutMillis - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); + CompletableFuture blocked = dynamicFilter.isBlocked(); + long timeLeft = max(dynamicFilteringWaitTimeoutMillis, dynamicFilter.getPreferredDynamicFilterTimeout().orElse(0)) - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); if (dynamicFilter.isAwaitable() && timeLeft > 0) { - return dynamicFilter.isBlocked() + return blocked .thenApply(_ -> EMPTY_BATCH) .completeOnTimeout(EMPTY_BATCH, timeLeft, MILLISECONDS); } 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 d13befb5d2df..eed3c2ef11de 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 @@ -41,6 +41,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -187,5 +188,11 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } } } diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java index 0f138d5ae673..2107331fb553 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeNodeLocalDynamicSplitPruning.java @@ -57,6 +57,7 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -381,6 +382,12 @@ public TupleDomain getCurrentPredicate() { return tupleDomain; } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0); + } }; } } diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java index 0d0395a0fb5e..700d3b13f7bd 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/BackgroundHiveSplitLoader.java @@ -69,6 +69,7 @@ import java.util.Map.Entry; import java.util.Optional; import java.util.OptionalInt; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; @@ -272,9 +273,10 @@ public TaskStatus process() // Block until one of below conditions is met: // 1. Completion of DynamicFilter // 2. Timeout after waiting for the configured time - long timeLeft = dynamicFilteringWaitTimeoutMillis - stopwatch.elapsed(MILLISECONDS); + CompletableFuture blocked = dynamicFilter.isBlocked(); + long timeLeft = Math.max(dynamicFilteringWaitTimeoutMillis, dynamicFilter.getPreferredDynamicFilterTimeout().orElse(0L)) - stopwatch.elapsed(MILLISECONDS); if (timeLeft > 0 && dynamicFilter.isAwaitable()) { - future = asVoid(toListenableFuture(dynamicFilter.isBlocked() + future = asVoid(toListenableFuture(blocked // As isBlocked() returns unmodifiableFuture, we need to create new future for correct propagation of the timeout .thenApply(Function.identity()) .orTimeout(timeLeft, MILLISECONDS))); 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 27f6563b24f7..6af7c86bc578 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 @@ -68,6 +68,7 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; @@ -334,6 +335,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } }, new Duration(1, SECONDS)); HiveSplitSource hiveSplitSource = hiveSplitSource(backgroundHiveSplitLoader); diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java index da8dd68b1d6f..340ab82366ed 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/TestNodeLocalDynamicSplitPruning.java @@ -40,6 +40,7 @@ import java.util.Map; import java.util.Optional; import java.util.OptionalInt; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -242,6 +243,12 @@ public TupleDomain getCurrentPredicate() { return tupleDomain; } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } }; } } 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 e61e3b36a5ef..7f1a3f7776ad 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 @@ -97,6 +97,7 @@ import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static java.lang.Math.clamp; +import static java.lang.Math.max; import static java.util.Collections.emptyIterator; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; @@ -208,9 +209,10 @@ public CompletableFuture getNextBatch(int maxSize) private CompletableFuture getNextBatchInternal(int maxSize) { - long timeLeft = dynamicFilteringWaitTimeoutMillis - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); + CompletableFuture blocked = dynamicFilter.isBlocked(); + long timeLeft = max(dynamicFilteringWaitTimeoutMillis, dynamicFilter.getPreferredDynamicFilterTimeout().orElse(0L)) - dynamicFilterWaitStopwatch.elapsed(MILLISECONDS); if (dynamicFilter.isAwaitable() && timeLeft > 0) { - return dynamicFilter.isBlocked() + return blocked .thenApply(_ -> EMPTY_BATCH) .completeOnTimeout(EMPTY_BATCH, timeLeft, MILLISECONDS); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 973bb82936ac..50fbf96eeb1a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -627,6 +627,12 @@ public TupleDomain getCurrentPredicate() { return tupleDomain; } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } }; } } 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 de96d8d8bd94..cff60ebfe364 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 @@ -202,6 +202,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } }, new Duration(2, SECONDS), alwaysTrue(), @@ -289,6 +295,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.empty(); + } }); assertThat(split.getFileStatisticsDomain()).isEqualTo(TupleDomain.withColumnDomains( ImmutableMap.of( 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 1c1f2efb0b8f..d198be17a2a7 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 @@ -42,6 +42,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -143,6 +144,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } } @Test 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 0c5e6bb2d054..11fcbe1221cc 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 @@ -27,7 +27,9 @@ import java.util.concurrent.CompletableFuture; import static io.trino.plugin.mongodb.MongoSessionProperties.getDynamicFilteringWaitTimeout; +import static java.lang.Math.max; import static java.util.Objects.requireNonNull; +import static java.util.concurrent.TimeUnit.MILLISECONDS; import static java.util.concurrent.TimeUnit.NANOSECONDS; public class MongoSplitManager @@ -75,10 +77,11 @@ public MongoSplitSource(ConnectorSession session, DynamicFilter dynamicFilter, C @Override public CompletableFuture getNextBatch(int maxSize) { - long remainingTimeoutNanos = getRemainingTimeoutNanos(); + CompletableFuture blocked = dynamicFilter.isBlocked(); + long remainingTimeoutNanos = getRemainingTimeoutNanos(dynamicFilter); if (remainingTimeoutNanos > 0 && dynamicFilter.isAwaitable()) { // wait for dynamic filter and yield - return dynamicFilter.isBlocked() + return blocked .thenApply(_ -> EMPTY_BATCH) .completeOnTimeout(EMPTY_BATCH, remainingTimeoutNanos, NANOSECONDS); } @@ -95,16 +98,16 @@ public void close() @Override public boolean isFinished() { - if (getRemainingTimeoutNanos() > 0 && dynamicFilter.isAwaitable()) { + if (getRemainingTimeoutNanos(dynamicFilter) > 0 && dynamicFilter.isAwaitable()) { return false; } return delegateSplitSource.isFinished(); } - private long getRemainingTimeoutNanos() + private long getRemainingTimeoutNanos(DynamicFilter dynamicFilter) { - return dynamicFilteringTimeoutNanos - (System.nanoTime() - startNanos); + return max(dynamicFilteringTimeoutNanos, MILLISECONDS.toNanos(dynamicFilter.getPreferredDynamicFilterTimeout().orElse(0L))) - (System.nanoTime() - startNanos); } } } diff --git a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoDynamicFiltering.java b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoDynamicFiltering.java index a4eb25223da4..6036b7e3ee33 100644 --- a/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoDynamicFiltering.java +++ b/plugin/trino-mongodb/src/test/java/io/trino/plugin/mongodb/TestMongoDynamicFiltering.java @@ -40,6 +40,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -140,6 +141,12 @@ public TupleDomain getCurrentPredicate() { return TupleDomain.all(); } + + @Override + public OptionalLong getPreferredDynamicFilterTimeout() + { + return OptionalLong.of(0L); + } } @Test diff --git a/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java b/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java index eee80f11535c..76cfefcc388a 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/AbstractDistributedEngineOnlyQueries.java @@ -183,6 +183,14 @@ public void testExplain() "Trino version: .*"); } + @Test + public void testExplainDynamicFilterInfo() + { + assertExplain( + "EXPLAIN SELECT * FROM nation a, nation b WHERE a.nationkey = b.nationkey", + "ScanFilter\\[.*, dynamicFilters = \\{nationkey = #df_\\d+ await\\[\\d+\\]}]"); + } + @Test public void testExplainDistributed() { @@ -227,7 +235,8 @@ public void testExplainAnalyzeDynamicFilterInfo() .setSystemProperty(ENABLE_LARGE_DYNAMIC_FILTERS, "true") .build(), "EXPLAIN ANALYZE SELECT * FROM nation a, nation b WHERE a.nationkey = b.nationkey", - "Dynamic filters: \n.*ranges=25, \\{\\[0], ..., \\[24]}.* collection time=\\d+.*")); + "Dynamic filters: \n.*ranges=25, \\{\\[0], ..., \\[24]}.* collection time=\\d+.*", + "ScanFilter\\[.*, dynamicFilters = \\{nationkey = #df_\\d+ await\\[\\d+\\]}]")); } @Test diff --git a/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java index e57ebef5fb5c..132377c881ef 100644 --- a/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java +++ b/testing/trino-tests/src/test/java/io/trino/sql/planner/BaseCostBasedPlanTest.java @@ -44,6 +44,7 @@ import java.io.UncheckedIOException; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.ArrayList; import java.util.List; import java.util.stream.IntStream; @@ -314,13 +315,34 @@ public Void visitAggregation(AggregationNode node, Integer indent) public Void visitFilter(FilterNode node, Integer indent) { DynamicFilters.ExtractResult filters = extractDynamicFilters(node.getPredicate()); + String unestimatableInputs = filters.getDynamicConjuncts().stream() + .filter(descriptor -> descriptor.getPreferredTimeout().isEmpty()) + .map(descriptor -> ((Reference) descriptor.getInput()).name() + "::" + descriptor.getOperator()) + .sorted() + .collect(joining(", ")); String inputs = filters.getDynamicConjuncts().stream() + .filter(descriptor -> descriptor.getPreferredTimeout().isPresent() && descriptor.getPreferredTimeout().getAsLong() == 0) + .map(descriptor -> ((Reference) descriptor.getInput()).name() + "::" + descriptor.getOperator()) + .sorted() + .collect(joining(", ")); + String awaitInputs = filters.getDynamicConjuncts().stream() + .filter(descriptor -> descriptor.getPreferredTimeout().isPresent() && descriptor.getPreferredTimeout().getAsLong() > 0) .map(descriptor -> ((Reference) descriptor.getInput()).name() + "::" + descriptor.getOperator()) .sorted() .collect(joining(", ")); - if (!inputs.isEmpty()) { - output(indent, "dynamic filter (%s)", inputs); + if (!inputs.isEmpty() || !awaitInputs.isEmpty() || !unestimatableInputs.isEmpty()) { + List msg = new ArrayList<>(); + if (!inputs.isEmpty()) { + msg.add("[%s]".formatted(inputs)); + } + if (!unestimatableInputs.isEmpty()) { + msg.add("unestimatable [%s]".formatted(unestimatableInputs)); + } + if (!awaitInputs.isEmpty()) { + msg.add("await [%s]".formatted(awaitInputs)); + } + output(indent, "dynamic filter (%s)", String.join(", ", msg)); indent = indent + 1; } return visitPlan(node, indent); diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q01.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q01.plan.txt index e37938554d90..5616640f7b82 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q01.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q01.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_store_sk]) partial aggregation over (sr_customer_sk, sr_store_sk) join (INNER, REPLICATED): - dynamic filter (sr_customer_sk::EQUAL, sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) + dynamic filter ([sr_customer_sk::EQUAL], await [sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -33,7 +33,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk_8, sr_store_sk_12]) partial aggregation over (sr_customer_sk_8, sr_store_sk_12) join (INNER, REPLICATED): - dynamic filter (sr_returned_date_sk_25::EQUAL) + dynamic filter (await [sr_returned_date_sk_25::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q02.plan.txt index ff679f60a2f0..51fda539343f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q02.plan.txt @@ -13,17 +13,17 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (d_day_name, d_week_seq) join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ws_sold_date_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL]) scan web_sales - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq::EQUAL, d_week_seq::EQUAL) + dynamic filter (unestimatable [d_week_seq::EQUAL], await [d_week_seq::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_week_seq_20]) - dynamic filter (d_week_seq_20::EQUAL) + dynamic filter (unestimatable [d_week_seq_20::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [operator_subtract]) @@ -38,13 +38,13 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (d_day_name_134, d_week_seq_124) join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ws_sold_date_sk_81::EQUAL) + dynamic filter (await [ws_sold_date_sk_81::EQUAL]) scan web_sales - dynamic filter (cs_sold_date_sk_117::EQUAL) + dynamic filter (await [cs_sold_date_sk_117::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq_124::EQUAL) + dynamic filter (await [d_week_seq_124::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_week_seq_168]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q03.plan.txt index 7a9948c1b370..1b8bc51bfca8 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q03.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q04.plan.txt index f9fdbfbb1f20..a964d4cc72c4 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q04.plan.txt @@ -15,11 +15,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk]) - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL) + dynamic filter (unestimatable [c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -33,11 +33,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_167]) - dynamic filter (ss_customer_sk_167::EQUAL, ss_sold_date_sk_187::EQUAL) + dynamic filter ([ss_customer_sk_167::EQUAL], await [ss_sold_date_sk_187::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_145]) - dynamic filter (c_customer_id_146::EQUAL, c_customer_id_146::EQUAL, c_customer_id_146::EQUAL, c_customer_id_146::EQUAL) + dynamic filter (unestimatable [c_customer_id_146::EQUAL, c_customer_id_146::EQUAL, c_customer_id_146::EQUAL, c_customer_id_146::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -51,11 +51,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_521]) - dynamic filter (cs_bill_customer_sk_521::EQUAL, cs_sold_date_sk_552::EQUAL) + dynamic filter ([cs_bill_customer_sk_521::EQUAL], await [cs_sold_date_sk_552::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_499]) - dynamic filter (c_customer_id_500::EQUAL, c_customer_id_500::EQUAL, c_customer_id_500::EQUAL) + dynamic filter (unestimatable [c_customer_id_500::EQUAL, c_customer_id_500::EQUAL, c_customer_id_500::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -69,11 +69,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_797]) - dynamic filter (cs_bill_customer_sk_797::EQUAL, cs_sold_date_sk_828::EQUAL) + dynamic filter ([cs_bill_customer_sk_797::EQUAL], await [cs_sold_date_sk_828::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_775]) - dynamic filter (c_customer_id_776::EQUAL, c_customer_id_776::EQUAL) + dynamic filter (unestimatable [c_customer_id_776::EQUAL, c_customer_id_776::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -87,11 +87,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1163]) - dynamic filter (ws_bill_customer_sk_1163::EQUAL, ws_sold_date_sk_1193::EQUAL) + dynamic filter ([ws_bill_customer_sk_1163::EQUAL], await [ws_sold_date_sk_1193::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_1140]) - dynamic filter (c_customer_id_1141::EQUAL) + dynamic filter (unestimatable [c_customer_id_1141::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -105,7 +105,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1439]) - dynamic filter (ws_bill_customer_sk_1439::EQUAL, ws_sold_date_sk_1469::EQUAL) + dynamic filter ([ws_bill_customer_sk_1439::EQUAL], await [ws_sold_date_sk_1469::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_1416]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q05.plan.txt index 0bfdc565b650..3c5ece21cbb2 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q05.plan.txt @@ -14,9 +14,9 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales - dynamic filter (sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) + dynamic filter (await [sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -31,9 +31,9 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (cs_catalog_page_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_catalog_page_sk::EQUAL, cs_sold_date_sk::EQUAL]) scan catalog_sales - dynamic filter (cr_catalog_page_sk::EQUAL, cr_returned_date_sk::EQUAL) + dynamic filter (await [cr_catalog_page_sk::EQUAL, cr_returned_date_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -48,15 +48,15 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): remote exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ws_sold_date_sk::EQUAL, ws_web_site_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL, ws_web_site_sk::EQUAL]) scan web_sales join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk_82, ws_order_number_96]) - dynamic filter (ws_item_sk_82::EQUAL, ws_order_number_96::EQUAL, ws_web_site_sk_92::EQUAL) + dynamic filter ([ws_item_sk_82::EQUAL, ws_order_number_96::EQUAL], await [ws_web_site_sk_92::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) - dynamic filter (wr_returned_date_sk::EQUAL) + dynamic filter (await [wr_returned_date_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q06.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q06.plan.txt index ad6f4aa1d560..fad440b891c0 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q06.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q06.plan.txt @@ -15,18 +15,18 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_customer_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_customer_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_month_seq::EQUAL) + dynamic filter (await [d_month_seq::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q07.plan.txt index 73b14e87af3a..4a1687a7566f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q07.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_cdemo_sk::EQUAL, ss_item_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_cdemo_sk::EQUAL, ss_item_sk::EQUAL], await [ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q08.plan.txt index 8b8ce6f9b4ec..1e382a9ae74b 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q08.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [substring]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -33,7 +33,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_zip_16) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ca_address_sk_7]) - dynamic filter (ca_address_sk_7::EQUAL) + dynamic filter ([ca_address_sk_7::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_current_addr_sk]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q10.plan.txt index 218823cc22d3..d0e024434289 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q10.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_customer_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL], unestimatable [c_customer_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) @@ -26,7 +26,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) partial aggregation over (ss_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -36,7 +36,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_sold_date_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -46,7 +46,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_ship_customer_sk]) partial aggregation over (cs_ship_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q11.plan.txt index a4111e246154..2c8e53f8b85c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q11.plan.txt @@ -13,11 +13,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk]) - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL) + dynamic filter (unestimatable [c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -31,11 +31,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_92]) - dynamic filter (ss_customer_sk_92::EQUAL, ss_sold_date_sk_112::EQUAL) + dynamic filter ([ss_customer_sk_92::EQUAL], await [ss_sold_date_sk_112::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_70]) - dynamic filter (c_customer_id_71::EQUAL, c_customer_id_71::EQUAL) + dynamic filter (unestimatable [c_customer_id_71::EQUAL, c_customer_id_71::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -49,11 +49,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_348]) - dynamic filter (ws_bill_customer_sk_348::EQUAL, ws_sold_date_sk_378::EQUAL) + dynamic filter ([ws_bill_customer_sk_348::EQUAL], await [ws_sold_date_sk_378::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_325]) - dynamic filter (c_customer_id_326::EQUAL) + dynamic filter (unestimatable [c_customer_id_326::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -67,7 +67,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_525]) - dynamic filter (ws_bill_customer_sk_525::EQUAL, ws_sold_date_sk_555::EQUAL) + dynamic filter ([ws_bill_customer_sk_525::EQUAL], await [ws_sold_date_sk_555::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_502]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q12.plan.txt index b04a36db4c75..3bd6f68d557a 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q12.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q13.plan.txt index 23d00c74fb55..0b1abf8645aa 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q13.plan.txt @@ -8,7 +8,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q14.plan.txt index 322e49120e33..dc98600deac8 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q14.plan.txt @@ -16,11 +16,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) + dynamic filter ([i_item_sk::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -36,11 +36,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_41::EQUAL, ss_sold_date_sk_62::EQUAL) + dynamic filter ([ss_item_sk_41::EQUAL], await [ss_sold_date_sk_62::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_72::EQUAL, i_category_id_76::EQUAL, i_class_id_74::EQUAL) + dynamic filter ([i_brand_id_72::EQUAL, i_category_id_76::EQUAL, i_class_id_74::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -49,11 +49,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_128::EQUAL, i_category_id_132::EQUAL, i_class_id_130::EQUAL) + dynamic filter ([i_brand_id_128::EQUAL, i_category_id_132::EQUAL, i_class_id_130::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -62,11 +62,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_184::EQUAL, i_category_id_188::EQUAL, i_class_id_186::EQUAL) + dynamic filter ([i_brand_id_184::EQUAL, i_category_id_188::EQUAL, i_class_id_186::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -81,21 +81,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_258::EQUAL) + dynamic filter (await [ss_sold_date_sk_258::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (cs_sold_date_sk_324::EQUAL) + dynamic filter (await [cs_sold_date_sk_324::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk_390::EQUAL) + dynamic filter (await [ws_sold_date_sk_390::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -109,11 +109,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_item_sk_440]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk_440::EQUAL, cs_item_sk_440::EQUAL, cs_sold_date_sk_459::EQUAL) + dynamic filter ([cs_item_sk_440::EQUAL, cs_item_sk_440::EQUAL], await [cs_sold_date_sk_459::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk_462::EQUAL) + dynamic filter ([i_item_sk_462::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -129,11 +129,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_549::EQUAL, ss_sold_date_sk_570::EQUAL) + dynamic filter ([ss_item_sk_549::EQUAL], await [ss_sold_date_sk_570::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_580::EQUAL, i_category_id_584::EQUAL, i_class_id_582::EQUAL) + dynamic filter ([i_brand_id_580::EQUAL, i_category_id_584::EQUAL, i_class_id_582::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -142,11 +142,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk_641::EQUAL, cs_sold_date_sk_660::EQUAL) + dynamic filter ([cs_item_sk_641::EQUAL], await [cs_sold_date_sk_660::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_670::EQUAL, i_category_id_674::EQUAL, i_class_id_672::EQUAL) + dynamic filter ([i_brand_id_670::EQUAL, i_category_id_674::EQUAL, i_class_id_672::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -155,11 +155,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk_719::EQUAL, ws_sold_date_sk_750::EQUAL) + dynamic filter ([ws_item_sk_719::EQUAL], await [ws_sold_date_sk_750::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_760::EQUAL, i_category_id_764::EQUAL, i_class_id_762::EQUAL) + dynamic filter ([i_brand_id_760::EQUAL, i_category_id_764::EQUAL, i_class_id_762::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -174,21 +174,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_838::EQUAL) + dynamic filter (await [ss_sold_date_sk_838::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (cs_sold_date_sk_904::EQUAL) + dynamic filter (await [cs_sold_date_sk_904::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk_970::EQUAL) + dynamic filter (await [ws_sold_date_sk_970::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -202,11 +202,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_item_sk_1008]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk_1008::EQUAL, ws_item_sk_1008::EQUAL, ws_sold_date_sk_1039::EQUAL) + dynamic filter ([ws_item_sk_1008::EQUAL, ws_item_sk_1008::EQUAL], await [ws_sold_date_sk_1039::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk_1042::EQUAL) + dynamic filter ([i_item_sk_1042::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -222,11 +222,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_1160, i_category_id_1164, i_class_id_1162) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_1129::EQUAL, ss_sold_date_sk_1150::EQUAL) + dynamic filter ([ss_item_sk_1129::EQUAL], await [ss_sold_date_sk_1150::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_1160::EQUAL, i_category_id_1164::EQUAL, i_class_id_1162::EQUAL) + dynamic filter ([i_brand_id_1160::EQUAL, i_category_id_1164::EQUAL, i_class_id_1162::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -235,11 +235,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_1250, i_category_id_1254, i_class_id_1252) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk_1221::EQUAL, cs_sold_date_sk_1240::EQUAL) + dynamic filter ([cs_item_sk_1221::EQUAL], await [cs_sold_date_sk_1240::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_1250::EQUAL, i_category_id_1254::EQUAL, i_class_id_1252::EQUAL) + dynamic filter ([i_brand_id_1250::EQUAL, i_category_id_1254::EQUAL, i_class_id_1252::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -248,11 +248,11 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand_id_1340, i_category_id_1344, i_class_id_1342) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk_1299::EQUAL, ws_sold_date_sk_1330::EQUAL) + dynamic filter ([ws_item_sk_1299::EQUAL], await [ws_sold_date_sk_1330::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_1340::EQUAL, i_category_id_1344::EQUAL, i_class_id_1342::EQUAL) + dynamic filter ([i_brand_id_1340::EQUAL, i_category_id_1344::EQUAL, i_class_id_1342::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -267,21 +267,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_1418::EQUAL) + dynamic filter (await [ss_sold_date_sk_1418::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (cs_sold_date_sk_1484::EQUAL) + dynamic filter (await [cs_sold_date_sk_1484::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim partial aggregation over () join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk_1550::EQUAL) + dynamic filter (await [ws_sold_date_sk_1550::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q15.plan.txt index 1959839c8b1e..0f699230b6e1 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q15.plan.txt @@ -11,11 +11,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_current_addr_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk]) - dynamic filter (cs_bill_customer_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_customer_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q16.plan.txt index d223997dcf33..068af758e5b6 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q16.plan.txt @@ -13,7 +13,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_call_center_sk::EQUAL, cs_order_number::EQUAL, cs_ship_addr_sk::EQUAL, cs_ship_date_sk::EQUAL) + dynamic filter ([cs_order_number::EQUAL, cs_ship_addr_sk::EQUAL], await [cs_call_center_sk::EQUAL, cs_ship_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q17.plan.txt index 3415b6d9c556..dc5813302a25 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q17.plan.txt @@ -15,15 +15,15 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) - dynamic filter (sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) + dynamic filter ([sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL], await [sr_returned_date_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_bill_customer_sk, cs_item_sk]) - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q18.plan.txt index e0b069101d56..addc52f11dc6 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q18.plan.txt @@ -14,14 +14,14 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk]) join (INNER, REPLICATED): - dynamic filter (cs_bill_cdemo_sk::EQUAL, cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_cdemo_sk::EQUAL, cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q19.plan.txt index c19aa5976cb9..35b8e47abd67 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q19.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -23,7 +23,7 @@ local exchange (GATHER, SINGLE, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q20.plan.txt index b497347bc142..38f67f592af9 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q20.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q21.plan.txt index e7d83fcf1a31..cc8f7be8e979 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q21.plan.txt @@ -9,7 +9,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL], await [inv_date_sk::EQUAL, inv_warehouse_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q22.plan.txt index afb08eba60d6..b677449434a3 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q22.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_brand_gid, i_category_gid, i_class_gid, i_product_name_gid) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL], await [inv_date_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q23.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q23.plan.txt index 6013775a2130..c7333a3b6ef6 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q23.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q23.plan.txt @@ -7,7 +7,7 @@ final aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk]) join (INNER, REPLICATED): - dynamic filter (cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter (unestimatable [cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -22,7 +22,7 @@ final aggregation over () partial aggregation over (d_date_6, gid, ss_item_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -37,7 +37,7 @@ final aggregation over () partial aggregation over (ss_customer_sk_43) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_43]) - dynamic filter (ss_customer_sk_43::EQUAL) + dynamic filter ([ss_customer_sk_43::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) @@ -54,7 +54,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_71]) - dynamic filter (ss_customer_sk_71::EQUAL, ss_sold_date_sk_91::EQUAL) + dynamic filter ([ss_customer_sk_71::EQUAL], await [ss_sold_date_sk_91::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_94]) @@ -68,7 +68,7 @@ final aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk]) join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter (unestimatable [ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -83,7 +83,7 @@ final aggregation over () partial aggregation over (d_date_210, gid_263, ss_item_sk_184) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_184::EQUAL, ss_sold_date_sk_205::EQUAL) + dynamic filter ([ss_item_sk_184::EQUAL], await [ss_sold_date_sk_205::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -98,7 +98,7 @@ final aggregation over () partial aggregation over (ss_customer_sk_271) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_271]) - dynamic filter (ss_customer_sk_271::EQUAL) + dynamic filter ([ss_customer_sk_271::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_294]) @@ -115,7 +115,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_318]) - dynamic filter (ss_customer_sk_318::EQUAL, ss_sold_date_sk_338::EQUAL) + dynamic filter ([ss_customer_sk_318::EQUAL], await [ss_sold_date_sk_338::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_341]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q24.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q24.plan.txt index 04812768637c..e0deda47f740 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q24.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q24.plan.txt @@ -18,22 +18,22 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_item_sk::EQUAL) + dynamic filter ([sr_item_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_zip::EQUAL) + dynamic filter ([s_zip::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_birth_country::EQUAL) + dynamic filter ([c_birth_country::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_zip, upper]) @@ -56,22 +56,22 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk_12, ss_ticket_number_19]) - dynamic filter (ss_customer_sk_13::EQUAL, ss_item_sk_12::EQUAL, ss_item_sk_12::EQUAL, ss_store_sk_17::EQUAL, ss_ticket_number_19::EQUAL) + dynamic filter ([ss_customer_sk_13::EQUAL, ss_item_sk_12::EQUAL, ss_item_sk_12::EQUAL, ss_ticket_number_19::EQUAL], await [ss_store_sk_17::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk_37, sr_ticket_number_44]) - dynamic filter (sr_item_sk_37::EQUAL) + dynamic filter ([sr_item_sk_37::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_zip_83::EQUAL) + dynamic filter ([s_zip_83::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_113]) - dynamic filter (c_birth_country_127::EQUAL) + dynamic filter ([c_birth_country_127::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_zip_142, upper_149]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q25.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q25.plan.txt index 43dba892575a..e700cc8f5514 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q25.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q25.plan.txt @@ -15,15 +15,15 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) - dynamic filter (sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) + dynamic filter ([sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL], await [sr_returned_date_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_bill_customer_sk, cs_item_sk]) - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q26.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q26.plan.txt index da4c893b03e8..81cf3516e1d6 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q26.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q26.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_bill_cdemo_sk::EQUAL, cs_item_sk::EQUAL, cs_promo_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_cdemo_sk::EQUAL, cs_item_sk::EQUAL], await [cs_promo_sk::EQUAL, cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q27.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q27.plan.txt index fdfa755cab91..a310383445dd 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q27.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q27.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_cdemo_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_cdemo_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q29.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q29.plan.txt index 43dba892575a..e700cc8f5514 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q29.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q29.plan.txt @@ -15,15 +15,15 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) - dynamic filter (sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) + dynamic filter ([sr_customer_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL], await [sr_returned_date_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_bill_customer_sk, cs_item_sk]) - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q30.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q30.plan.txt index 6287685a00c6..e858edab0621 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q30.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q30.plan.txt @@ -14,7 +14,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state, wr_returning_customer_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (wr_returned_date_sk::EQUAL, wr_returning_addr_sk::EQUAL, wr_returning_customer_sk::EQUAL) + dynamic filter ([wr_returning_addr_sk::EQUAL, wr_returning_customer_sk::EQUAL], await [wr_returned_date_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -39,7 +39,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state_85, wr_returning_customer_sk_27) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (wr_returned_date_sk_44::EQUAL, wr_returning_addr_sk_30::EQUAL) + dynamic filter ([wr_returning_addr_sk_30::EQUAL], await [wr_returned_date_sk_44::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q31.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q31.plan.txt index f27dbfecd82d..0555d8380ed7 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q31.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q31.plan.txt @@ -13,14 +13,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) - dynamic filter (ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL) + dynamic filter (unestimatable [ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL, ca_county::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [1, 2000, ca_county_66]) @@ -31,14 +31,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk_9]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk_9::EQUAL, ss_sold_date_sk_26::EQUAL) + dynamic filter ([ss_addr_sk_9::EQUAL], await [ss_sold_date_sk_26::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_59]) - dynamic filter (ca_county_66::EQUAL, ca_county_66::EQUAL, ca_county_66::EQUAL, ca_county_66::EQUAL) + dynamic filter (unestimatable [ca_county_66::EQUAL, ca_county_66::EQUAL, ca_county_66::EQUAL, ca_county_66::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [1, 2000, ca_county_137]) @@ -49,14 +49,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk_80]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk_80::EQUAL, ss_sold_date_sk_97::EQUAL) + dynamic filter ([ss_addr_sk_80::EQUAL], await [ss_sold_date_sk_97::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_130]) - dynamic filter (ca_county_137::EQUAL, ca_county_137::EQUAL, ca_county_137::EQUAL) + dynamic filter (unestimatable [ca_county_137::EQUAL, ca_county_137::EQUAL, ca_county_137::EQUAL]) scan customer_address final aggregation over (ca_county_185, d_qoy_158, d_year_154) local exchange (GATHER, SINGLE, []) @@ -65,14 +65,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_addr_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_178]) - dynamic filter (ca_county_185::EQUAL, ca_county_185::EQUAL) + dynamic filter (unestimatable [ca_county_185::EQUAL, ca_county_185::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [1, 2000, ca_county_267]) @@ -83,14 +83,14 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk_200]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk_200::EQUAL, ws_sold_date_sk_227::EQUAL) + dynamic filter ([ws_bill_addr_sk_200::EQUAL], await [ws_sold_date_sk_227::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_260]) - dynamic filter (ca_county_267::EQUAL) + dynamic filter (unestimatable [ca_county_267::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [1, 2000, ca_county_349]) @@ -101,7 +101,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk_282]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk_282::EQUAL, ws_sold_date_sk_309::EQUAL) + dynamic filter ([ws_bill_addr_sk_282::EQUAL], await [ws_sold_date_sk_309::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q32.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q32.plan.txt index f8db856879fd..a5482042c927 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q32.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q32.plan.txt @@ -7,7 +7,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [cs_item_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -20,7 +20,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [cs_item_sk_18]) partial aggregation over (cs_item_sk_18) join (INNER, REPLICATED): - dynamic filter (cs_sold_date_sk_37::EQUAL) + dynamic filter (await [cs_sold_date_sk_37::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q33.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q33.plan.txt index 4964eee9a6bb..7d75599f1f86 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q33.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q33.plan.txt @@ -14,7 +14,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_manufact_id_6::EQUAL) + dynamic filter ([i_manufact_id_6::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -43,7 +43,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -53,7 +53,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_manufact_id_95::EQUAL) + dynamic filter ([i_manufact_id_95::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -72,7 +72,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -82,7 +82,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_manufact_id_195::EQUAL) + dynamic filter ([i_manufact_id_195::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q34.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q34.plan.txt index 50c177f648c8..d113671bd728 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q34.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q34.plan.txt @@ -10,7 +10,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q35.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q35.plan.txt index 9c011616ceb2..422b2aa7244c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q35.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q35.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_customer_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL], unestimatable [c_customer_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) @@ -26,7 +26,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) partial aggregation over (ss_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -36,7 +36,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_sold_date_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -46,7 +46,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_ship_customer_sk]) partial aggregation over (cs_ship_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q36.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q36.plan.txt index 9b41524e48df..77872b48cbdb 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q36.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q36.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q37.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q37.plan.txt index 5718f62fc120..8bd4ebb6f14d 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q37.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q37.plan.txt @@ -10,11 +10,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [inv_item_sk]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_item_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL, inv_item_sk::EQUAL], await [inv_date_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) + dynamic filter ([i_item_sk::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q38.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q38.plan.txt index 95ea593af612..a1f440be80a2 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q38.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q38.plan.txt @@ -12,7 +12,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ss_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -28,7 +28,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_bill_customer_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_customer_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -44,7 +44,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_bill_customer_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_customer_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q39.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q39.plan.txt index 0a0531101462..73e9ef7343fa 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q39.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q39.plan.txt @@ -10,15 +10,15 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL, inv_warehouse_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL], unestimatable [inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL], await [inv_date_sk::EQUAL, inv_warehouse_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) + dynamic filter (unestimatable [i_item_sk::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (w_warehouse_sk::EQUAL) + dynamic filter (unestimatable [w_warehouse_sk::EQUAL]) scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -32,7 +32,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (inv_date_sk_11::EQUAL, inv_item_sk_8::EQUAL, inv_warehouse_sk_9::EQUAL) + dynamic filter ([inv_item_sk_8::EQUAL], await [inv_date_sk_11::EQUAL, inv_warehouse_sk_9::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q40.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q40.plan.txt index a3f9b689189b..6a88bb9be4fc 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q40.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q40.plan.txt @@ -11,11 +11,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL, cs_warehouse_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL, cs_warehouse_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - dynamic filter (cr_item_sk::EQUAL) + dynamic filter ([cr_item_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q41.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q41.plan.txt index 9b5384cad7b9..a320578c7ad8 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q41.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q41.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_manufact_14, i_manufact_id, i_product_name, unique) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [i_manufact_14]) - dynamic filter (i_manufact_14::EQUAL) + dynamic filter ([i_manufact_14::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_manufact]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q42.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q42.plan.txt index 7d3337e7ba1a..bc511646f855 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q42.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q42.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_year, i_category, i_category_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q43.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q43.plan.txt index eae698fcf07b..ce44e36e8c75 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q43.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q43.plan.txt @@ -12,7 +12,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_day_name, s_store_id, s_store_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q45.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q45.plan.txt index aa1ddd9bf716..171c6da18993 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q45.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q45.plan.txt @@ -13,11 +13,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_current_addr_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) - dynamic filter (ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q46.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q46.plan.txt index f4441392f482..ada922053b78 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q46.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q46.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_customer_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -29,7 +29,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q47.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q47.plan.txt index deb4d70f5276..ee5f7710106c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q47.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q47.plan.txt @@ -13,18 +13,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand::EQUAL, i_brand::EQUAL, i_category::EQUAL, i_category::EQUAL) + dynamic filter (unestimatable [i_brand::EQUAL, i_brand::EQUAL, i_category::EQUAL, i_category::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_company_name::EQUAL, s_company_name::EQUAL, s_store_name::EQUAL, s_store_name::EQUAL) + dynamic filter (unestimatable [s_company_name::EQUAL, s_company_name::EQUAL, s_store_name::EQUAL, s_store_name::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_14, i_category_18, s_company_name_102, s_store_name_90]) @@ -35,18 +35,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_31::EQUAL, ss_sold_date_sk_52::EQUAL, ss_store_sk_36::EQUAL) + dynamic filter ([ss_item_sk_31::EQUAL], await [ss_sold_date_sk_52::EQUAL, ss_store_sk_36::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_14::EQUAL, i_category_18::EQUAL) + dynamic filter (unestimatable [i_brand_14::EQUAL, i_category_18::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_company_name_102::EQUAL, s_store_name_90::EQUAL) + dynamic filter (unestimatable [s_company_name_102::EQUAL, s_store_name_90::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_127, i_category_131, s_company_name_215, s_store_name_203]) @@ -57,7 +57,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk_144::EQUAL, ss_sold_date_sk_165::EQUAL, ss_store_sk_149::EQUAL) + dynamic filter ([ss_item_sk_144::EQUAL], await [ss_sold_date_sk_165::EQUAL, ss_store_sk_149::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q48.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q48.plan.txt index 34c419a96b08..64cb3a5862d7 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q48.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q48.plan.txt @@ -7,7 +7,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q49.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q49.plan.txt index 39797e5a0d75..ba9cdc18dc40 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q49.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q49.plan.txt @@ -15,7 +15,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) - dynamic filter (ws_item_sk::EQUAL, ws_order_number::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL, ws_order_number::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) @@ -34,7 +34,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_item_sk::EQUAL, cs_order_number::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL, cs_order_number::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) @@ -53,7 +53,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q50.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q50.plan.txt index 9fceb4963208..14398a4394fd 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q50.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q50.plan.txt @@ -11,11 +11,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) - dynamic filter (sr_returned_date_sk::EQUAL) + dynamic filter (await [sr_returned_date_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q51.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q51.plan.txt index 731195824eec..9f005a55c27f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q51.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q51.plan.txt @@ -12,7 +12,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date, ws_item_sk]) partial aggregation over (d_date, ws_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_7, ss_item_sk]) partial aggregation over (d_date_7, ss_item_sk) join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q52.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q52.plan.txt index 7a9948c1b370..1b8bc51bfca8 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q52.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q52.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_year, i_brand, i_brand_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q53.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q53.plan.txt index 88551fd7e11f..afc81c472524 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q53.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q53.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q54.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q54.plan.txt index e9899dbd1f5d..aa1480ce2329 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q54.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q54.plan.txt @@ -25,9 +25,9 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (cs_bill_customer_sk::EQUAL, cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_customer_sk::EQUAL, cs_bill_customer_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales - dynamic filter (ws_bill_customer_sk::EQUAL, ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_customer_sk::EQUAL, ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -37,22 +37,22 @@ local exchange (GATHER, SINGLE, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_customer_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_customer_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) - dynamic filter (ca_county::EQUAL, ca_state::EQUAL) + dynamic filter (await [ca_county::EQUAL, ca_state::EQUAL]) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_month_seq_17::LESS_THAN_OR_EQUAL) + dynamic filter (await [d_month_seq_17::LESS_THAN_OR_EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q55.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q55.plan.txt index f7d27cdedf5f..b9507ab07dcb 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q55.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q55.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q56.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q56.plan.txt index 30e04d53aa66..40cc1e6fc443 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q56.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q56.plan.txt @@ -14,7 +14,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_6::EQUAL) + dynamic filter ([i_item_id_6::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -43,7 +43,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -53,7 +53,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_83::EQUAL) + dynamic filter ([i_item_id_83::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -72,7 +72,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -82,7 +82,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_183::EQUAL) + dynamic filter ([i_item_id_183::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q57.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q57.plan.txt index a68812d48435..6479a463a79d 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q57.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q57.plan.txt @@ -13,18 +13,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_call_center_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_call_center_sk::EQUAL, cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand::EQUAL, i_brand::EQUAL, i_category::EQUAL, i_category::EQUAL) + dynamic filter (unestimatable [i_brand::EQUAL, i_brand::EQUAL, i_category::EQUAL, i_category::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (cc_name::EQUAL, cc_name::EQUAL) + dynamic filter (unestimatable [cc_name::EQUAL, cc_name::EQUAL]) scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name_102, i_brand_14, i_category_18]) @@ -35,18 +35,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_call_center_sk_40::EQUAL, cs_item_sk_44::EQUAL, cs_sold_date_sk_63::EQUAL) + dynamic filter ([cs_item_sk_44::EQUAL], await [cs_call_center_sk_40::EQUAL, cs_sold_date_sk_63::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_14::EQUAL, i_category_18::EQUAL) + dynamic filter (unestimatable [i_brand_14::EQUAL, i_category_18::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (cc_name_102::EQUAL) + dynamic filter (unestimatable [cc_name_102::EQUAL]) scan call_center local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name_227, i_brand_139, i_category_143]) @@ -57,7 +57,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_call_center_sk_165::EQUAL, cs_item_sk_169::EQUAL, cs_sold_date_sk_188::EQUAL) + dynamic filter ([cs_item_sk_169::EQUAL], await [cs_call_center_sk_165::EQUAL, cs_sold_date_sk_188::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q58.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q58.plan.txt index a803336da714..2c17698fdc4c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q58.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q58.plan.txt @@ -12,22 +12,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id::EQUAL, i_item_id::EQUAL) + dynamic filter (unestimatable [i_item_id::EQUAL, i_item_id::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date::EQUAL) + dynamic filter (unestimatable [d_date::EQUAL]) scan date_dim final aggregation over (d_date_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_6]) partial aggregation over (d_date_6) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_8::EQUAL) + dynamic filter (await [d_week_seq_8::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -42,22 +42,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_94]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_69::EQUAL) + dynamic filter (unestimatable [i_item_id_69::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date_94::EQUAL) + dynamic filter (unestimatable [d_date_94::EQUAL]) scan date_dim final aggregation over (d_date_126) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_126]) partial aggregation over (d_date_126) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_128::EQUAL) + dynamic filter (await [d_week_seq_128::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -72,21 +72,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_215]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date_215::EQUAL) + dynamic filter (unestimatable [d_date_215::EQUAL]) scan date_dim final aggregation over (d_date_247) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_247]) partial aggregation over (d_date_247) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_249::EQUAL) + dynamic filter (await [d_week_seq_249::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q59.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q59.plan.txt index d969d0e6e7ac..68045c2b57af 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q59.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q59.plan.txt @@ -15,19 +15,19 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_day_name, d_week_seq, ss_store_sk]) partial aggregation over (d_day_name, d_week_seq, ss_store_sk) join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq::EQUAL, d_week_seq::EQUAL) + dynamic filter (unestimatable [d_week_seq::EQUAL], await [d_week_seq::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_store_id::EQUAL) + dynamic filter (unestimatable [s_store_id::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq_20::EQUAL) + dynamic filter (unestimatable [d_week_seq_20::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [operator_subtract, s_store_id_116]) @@ -42,11 +42,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_day_name_85, d_week_seq_75, ss_store_sk_52]) partial aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_68::EQUAL, ss_store_sk_52::EQUAL) + dynamic filter (await [ss_sold_date_sk_68::EQUAL, ss_store_sk_52::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq_75::EQUAL) + dynamic filter (await [d_week_seq_75::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q60.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q60.plan.txt index 30e04d53aa66..40cc1e6fc443 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q60.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q60.plan.txt @@ -14,7 +14,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_6::EQUAL) + dynamic filter ([i_item_id_6::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -43,7 +43,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -53,7 +53,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_83::EQUAL) + dynamic filter ([i_item_id_83::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -72,7 +72,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_addr_sk]) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -82,7 +82,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_183::EQUAL) + dynamic filter ([i_item_id_183::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q61.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q61.plan.txt index 9fc7aac337ab..e5ca041d2353 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q61.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q61.plan.txt @@ -11,7 +11,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_item_sk::EQUAL], await [ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ cross join: scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) @@ -43,7 +43,7 @@ cross join: remote exchange (REPARTITION, HASH, [ss_customer_sk_14]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk_14::EQUAL, ss_item_sk_13::EQUAL, ss_sold_date_sk_34::EQUAL, ss_store_sk_18::EQUAL) + dynamic filter ([ss_customer_sk_14::EQUAL, ss_item_sk_13::EQUAL], await [ss_sold_date_sk_34::EQUAL, ss_store_sk_18::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -53,7 +53,7 @@ cross join: scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_98]) - dynamic filter (c_current_addr_sk_102::EQUAL) + dynamic filter ([c_current_addr_sk_102::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_118]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q62.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q62.plan.txt index cdad170ba430..3d0e330edf7a 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q62.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q62.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_ship_date_sk::EQUAL, ws_ship_mode_sk::EQUAL, ws_warehouse_sk::EQUAL, ws_web_site_sk::EQUAL) + dynamic filter (await [ws_ship_date_sk::EQUAL, ws_ship_mode_sk::EQUAL, ws_warehouse_sk::EQUAL, ws_web_site_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q63.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q63.plan.txt index d32d6bebb82f..be6cc3be1d54 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q63.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q63.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q64.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q64.plan.txt index 00e9dbc92e67..abbe2c6a88d4 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q64.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q64.plan.txt @@ -29,11 +29,11 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL, ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_cdemo_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_item_sk::EQUAL, ss_ticket_number::EQUAL], await [ss_hdemo_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_item_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL) + dynamic filter ([sr_item_sk::EQUAL, sr_item_sk::EQUAL, sr_item_sk::EQUAL]) scan store_returns final aggregation over (cs_item_sk) local exchange (GATHER, SINGLE, []) @@ -41,22 +41,22 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (cs_item_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_order_number::EQUAL) + dynamic filter ([cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_order_number::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - dynamic filter (cr_item_sk::EQUAL, cr_item_sk::EQUAL) + dynamic filter ([cr_item_sk::EQUAL, cr_item_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_store_name::EQUAL, s_zip::EQUAL) + dynamic filter ([s_store_name::EQUAL, s_zip::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_current_hdemo_sk::EQUAL, c_first_sales_date_sk::EQUAL, c_first_shipto_date_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL], await [c_current_hdemo_sk::EQUAL, c_first_sales_date_sk::EQUAL, c_first_shipto_date_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -75,11 +75,11 @@ remote exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (hd_income_band_sk::EQUAL) + dynamic filter (await [hd_income_band_sk::EQUAL]) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (hd_income_band_sk_91::EQUAL) + dynamic filter (await [hd_income_band_sk_91::EQUAL]) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) @@ -95,7 +95,7 @@ remote exchange (GATHER, SINGLE, []) scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) + dynamic filter ([i_item_sk::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_store_name_336, s_zip_356, ss_item_sk_127]) @@ -125,11 +125,11 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk_127]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk_127, ss_ticket_number_134]) - dynamic filter (ss_addr_sk_131::EQUAL, ss_cdemo_sk_129::EQUAL, ss_customer_sk_128::EQUAL, ss_hdemo_sk_130::EQUAL, ss_item_sk_127::EQUAL, ss_item_sk_127::EQUAL, ss_item_sk_127::EQUAL, ss_promo_sk_133::EQUAL, ss_sold_date_sk_148::EQUAL, ss_store_sk_132::EQUAL, ss_ticket_number_134::EQUAL) + dynamic filter ([ss_addr_sk_131::EQUAL, ss_cdemo_sk_129::EQUAL, ss_customer_sk_128::EQUAL, ss_item_sk_127::EQUAL, ss_item_sk_127::EQUAL, ss_item_sk_127::EQUAL, ss_ticket_number_134::EQUAL], await [ss_hdemo_sk_130::EQUAL, ss_promo_sk_133::EQUAL, ss_sold_date_sk_148::EQUAL, ss_store_sk_132::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk_152, sr_ticket_number_159]) - dynamic filter (sr_item_sk_152::EQUAL, sr_item_sk_152::EQUAL) + dynamic filter ([sr_item_sk_152::EQUAL, sr_item_sk_152::EQUAL]) scan store_returns final aggregation over (cs_item_sk_187) local exchange (GATHER, SINGLE, []) @@ -137,11 +137,11 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (cs_item_sk_187) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk_187, cs_order_number_189]) - dynamic filter (cs_item_sk_187::EQUAL, cs_item_sk_187::EQUAL, cs_order_number_189::EQUAL) + dynamic filter ([cs_item_sk_187::EQUAL, cs_item_sk_187::EQUAL, cs_order_number_189::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk_210, cr_order_number_224]) - dynamic filter (cr_item_sk_210::EQUAL) + dynamic filter ([cr_item_sk_210::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -151,7 +151,7 @@ remote exchange (GATHER, SINGLE, []) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_362]) - dynamic filter (c_current_addr_sk_366::EQUAL, c_current_cdemo_sk_364::EQUAL, c_current_hdemo_sk_365::EQUAL, c_first_sales_date_sk_368::EQUAL, c_first_shipto_date_sk_367::EQUAL) + dynamic filter ([c_current_addr_sk_366::EQUAL, c_current_cdemo_sk_364::EQUAL], await [c_current_hdemo_sk_365::EQUAL, c_first_sales_date_sk_368::EQUAL, c_first_shipto_date_sk_367::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -170,11 +170,11 @@ remote exchange (GATHER, SINGLE, []) scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (hd_income_band_sk_426::EQUAL) + dynamic filter (await [hd_income_band_sk_426::EQUAL]) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (hd_income_band_sk_433::EQUAL) + dynamic filter (await [hd_income_band_sk_433::EQUAL]) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_439]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q65.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q65.plan.txt index b77e321c37ca..df8f4dc6c209 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q65.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q65.plan.txt @@ -14,14 +14,14 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk, ss_store_sk]) partial aggregation over (ss_item_sk, ss_store_sk) join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (unestimatable [ss_store_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_store_sk]) - dynamic filter (s_store_sk::EQUAL) + dynamic filter (unestimatable [s_store_sk::EQUAL]) scan store local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_store_sk_12]) @@ -30,7 +30,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk_7, ss_store_sk_12]) partial aggregation over (ss_item_sk_7, ss_store_sk_12) join (INNER, REPLICATED): - dynamic filter (ss_item_sk_7::EQUAL, ss_sold_date_sk_28::EQUAL) + dynamic filter ([ss_item_sk_7::EQUAL], await [ss_sold_date_sk_28::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q66.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q66.plan.txt index 0efc0c1b56aa..a31944604f83 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q66.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q66.plan.txt @@ -17,7 +17,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_ship_mode_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_sold_time_sk::EQUAL, ws_warehouse_sk::EQUAL) + dynamic filter (await [ws_ship_mode_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_sold_time_sk::EQUAL, ws_warehouse_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -44,7 +44,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_ship_mode_sk::EQUAL, cs_sold_date_sk::EQUAL, cs_sold_time_sk::EQUAL, cs_warehouse_sk::EQUAL) + dynamic filter (await [cs_ship_mode_sk::EQUAL, cs_sold_date_sk::EQUAL, cs_sold_time_sk::EQUAL, cs_warehouse_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q67.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q67.plan.txt index cfcd8e2e359e..f030200be681 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q67.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q67.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q68.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q68.plan.txt index f4441392f482..ada922053b78 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q68.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q68.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_addr_sk::EQUAL, ss_customer_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -29,7 +29,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q69.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q69.plan.txt index 8222988cd67b..3ee474f724d1 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q69.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q69.plan.txt @@ -13,7 +13,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_customer_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL], unestimatable [c_customer_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) @@ -26,7 +26,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) partial aggregation over (ss_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -36,7 +36,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) partial aggregation over (ws_bill_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_sold_date_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -46,7 +46,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_ship_customer_sk]) partial aggregation over (cs_ship_customer_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q70.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q70.plan.txt index 2ebf58513664..d052244e538f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q70.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q70.plan.txt @@ -12,14 +12,14 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_state]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_state::EQUAL) + dynamic filter (unestimatable [s_state::EQUAL]) scan store single aggregation over (s_state_53) final aggregation over (s_state_53) @@ -28,7 +28,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (s_state_53) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_26::EQUAL, ss_store_sk_10::EQUAL) + dynamic filter (await [ss_sold_date_sk_26::EQUAL, ss_store_sk_10::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q71.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q71.plan.txt index 931a3be81bef..79145937def9 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q71.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q71.plan.txt @@ -9,19 +9,19 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_sold_time_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL, ws_sold_time_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL, cs_sold_time_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL, cs_sold_time_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_sold_time_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_sold_time_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q72.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q72.plan.txt index e30e9f489442..4ecfb5901a49 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q72.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q72.plan.txt @@ -17,11 +17,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk]) - dynamic filter (cs_bill_cdemo_sk::EQUAL, cs_bill_hdemo_sk::EQUAL, cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_quantity::GREATER_THAN, cs_ship_date_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_cdemo_sk::EQUAL, cs_item_sk::EQUAL, cs_item_sk::EQUAL, cs_quantity::GREATER_THAN], await [cs_bill_hdemo_sk::EQUAL, cs_ship_date_sk::EQUAL, cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [inv_item_sk]) - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL], await [inv_date_sk::EQUAL, inv_warehouse_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -37,7 +37,7 @@ local exchange (GATHER, SINGLE, []) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_week_seq::EQUAL) + dynamic filter (await [d_week_seq::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q73.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q73.plan.txt index 50c177f648c8..d113671bd728 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q73.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q73.plan.txt @@ -10,7 +10,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q74.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q74.plan.txt index fbba72c81538..8110950c87a4 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q74.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q74.plan.txt @@ -13,11 +13,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk]) - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL) + dynamic filter (unestimatable [c_customer_id::EQUAL, c_customer_id::EQUAL, c_customer_id::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -31,11 +31,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_86]) - dynamic filter (ss_customer_sk_86::EQUAL, ss_sold_date_sk_106::EQUAL) + dynamic filter ([ss_customer_sk_86::EQUAL], await [ss_sold_date_sk_106::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_64]) - dynamic filter (c_customer_id_65::EQUAL, c_customer_id_65::EQUAL) + dynamic filter (unestimatable [c_customer_id_65::EQUAL, c_customer_id_65::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -49,11 +49,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_335]) - dynamic filter (ws_bill_customer_sk_335::EQUAL, ws_sold_date_sk_365::EQUAL) + dynamic filter ([ws_bill_customer_sk_335::EQUAL], await [ws_sold_date_sk_365::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_312]) - dynamic filter (c_customer_id_313::EQUAL) + dynamic filter (unestimatable [c_customer_id_313::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -67,7 +67,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_506]) - dynamic filter (ws_bill_customer_sk_506::EQUAL, ws_sold_date_sk_536::EQUAL) + dynamic filter ([ws_bill_customer_sk_506::EQUAL], await [ws_sold_date_sk_536::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_483]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q75.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q75.plan.txt index ff1fdec81f85..6f1c06f20ec9 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q75.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q75.plan.txt @@ -12,11 +12,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_7::EQUAL, i_category_id_9::EQUAL, i_class_id_8::EQUAL, i_manufact_id_10::EQUAL) + dynamic filter ([i_brand_id_7::EQUAL, i_category_id_9::EQUAL, i_class_id_8::EQUAL, i_manufact_id_10::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -30,11 +30,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_28::EQUAL, i_category_id_32::EQUAL, i_class_id_30::EQUAL, i_manufact_id_34::EQUAL) + dynamic filter ([i_brand_id_28::EQUAL, i_category_id_32::EQUAL, i_class_id_30::EQUAL, i_manufact_id_34::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -48,11 +48,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_brand_id_88::EQUAL, i_category_id_92::EQUAL, i_class_id_90::EQUAL, i_manufact_id_94::EQUAL) + dynamic filter ([i_brand_id_88::EQUAL, i_category_id_92::EQUAL, i_class_id_90::EQUAL, i_manufact_id_94::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -69,7 +69,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_item_sk_169, cs_order_number_171]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_item_sk_169::EQUAL, cs_sold_date_sk_188::EQUAL) + dynamic filter ([cs_item_sk_169::EQUAL], await [cs_sold_date_sk_188::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -86,7 +86,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk_277, ss_ticket_number_284]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_item_sk_277::EQUAL, ss_sold_date_sk_298::EQUAL) + dynamic filter ([ss_item_sk_277::EQUAL], await [ss_sold_date_sk_298::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -103,7 +103,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_item_sk_381, ws_order_number_395]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_item_sk_381::EQUAL, ws_sold_date_sk_412::EQUAL) + dynamic filter ([ws_item_sk_381::EQUAL], await [ws_sold_date_sk_412::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q76.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q76.plan.txt index a8a1308b5e81..7a74d27954fc 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q76.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q76.plan.txt @@ -8,7 +8,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -21,7 +21,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk]) - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_sk_17]) @@ -33,7 +33,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q77.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q77.plan.txt index 0dbbf5271c8e..a1087cc56219 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q77.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q77.plan.txt @@ -14,7 +14,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ss_store_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -28,7 +28,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (sr_store_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) + dynamic filter (await [sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -42,7 +42,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_call_center_sk]) partial aggregation over (cs_call_center_sk) join (INNER, REPLICATED): - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -54,7 +54,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_call_center_sk]) partial aggregation over (cr_call_center_sk) join (INNER, REPLICATED): - dynamic filter (cr_returned_date_sk::EQUAL) + dynamic filter (await [cr_returned_date_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -66,7 +66,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk::EQUAL, ws_web_page_sk::EQUAL) + dynamic filter (await [ws_sold_date_sk::EQUAL, ws_web_page_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -80,7 +80,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (wr_returned_date_sk::EQUAL, wr_web_page_sk::EQUAL) + dynamic filter (await [wr_returned_date_sk::EQUAL, wr_web_page_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q78.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q78.plan.txt index e86c08bc4360..f363f284d785 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q78.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q78.plan.txt @@ -12,11 +12,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL, ss_customer_sk::EQUAL, ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_item_sk::EQUAL) + dynamic filter ([sr_item_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -28,7 +28,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) - dynamic filter (ws_bill_customer_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_customer_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) @@ -45,7 +45,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q79.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q79.plan.txt index f8e1f2875516..58130d9a6ba2 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q79.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q79.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q80.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q80.plan.txt index e703b7384053..1cd6ee4df686 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q80.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q80.plan.txt @@ -17,11 +17,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_item_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_item_sk::EQUAL) + dynamic filter ([sr_item_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -45,11 +45,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_catalog_page_sk::EQUAL, cs_item_sk::EQUAL, cs_promo_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_item_sk::EQUAL], await [cs_catalog_page_sk::EQUAL, cs_promo_sk::EQUAL, cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - dynamic filter (cr_item_sk::EQUAL) + dynamic filter ([cr_item_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -73,11 +73,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) - dynamic filter (ws_item_sk::EQUAL, ws_promo_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_web_site_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_promo_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_web_site_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) - dynamic filter (wr_item_sk::EQUAL) + dynamic filter ([wr_item_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q81.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q81.plan.txt index 257e656cb546..b2ed6c6383a0 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q81.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q81.plan.txt @@ -15,7 +15,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state, cr_returning_customer_sk) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cr_returned_date_sk::EQUAL, cr_returning_addr_sk::EQUAL, cr_returning_customer_sk::EQUAL) + dynamic filter ([cr_returning_addr_sk::EQUAL, cr_returning_customer_sk::EQUAL], await [cr_returned_date_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -25,7 +25,7 @@ local exchange (GATHER, SINGLE, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_4]) @@ -40,7 +40,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (ca_state_88, cr_returning_customer_sk_27) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cr_returned_date_sk_47::EQUAL, cr_returning_addr_sk_30::EQUAL) + dynamic filter ([cr_returning_addr_sk_30::EQUAL], await [cr_returned_date_sk_47::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q82.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q82.plan.txt index 819df4dd5106..cbb4d674240f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q82.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q82.plan.txt @@ -10,11 +10,11 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [inv_item_sk]) join (INNER, REPLICATED, can skip output duplicates): join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_item_sk::EQUAL) + dynamic filter ([inv_item_sk::EQUAL, inv_item_sk::EQUAL], await [inv_date_sk::EQUAL]) scan inventory local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) + dynamic filter ([i_item_sk::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q83.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q83.plan.txt index bc8a7744991d..3fed2dbadb70 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q83.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q83.plan.txt @@ -12,22 +12,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) + dynamic filter ([sr_item_sk::EQUAL], await [sr_returned_date_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id::EQUAL, i_item_id::EQUAL) + dynamic filter (unestimatable [i_item_id::EQUAL, i_item_id::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date::EQUAL) + dynamic filter (unestimatable [d_date::EQUAL]) scan date_dim final aggregation over (d_date_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_6]) partial aggregation over (d_date_6) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_8::EQUAL) + dynamic filter (await [d_week_seq_8::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -44,22 +44,22 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_99]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cr_item_sk::EQUAL, cr_returned_date_sk::EQUAL) + dynamic filter ([cr_item_sk::EQUAL], await [cr_returned_date_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_74::EQUAL) + dynamic filter (unestimatable [i_item_id_74::EQUAL]) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date_99::EQUAL) + dynamic filter (unestimatable [d_date_99::EQUAL]) scan date_dim final aggregation over (d_date_131) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_131]) partial aggregation over (d_date_131) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_133::EQUAL) + dynamic filter (await [d_week_seq_133::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -76,21 +76,21 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_225]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (wr_item_sk::EQUAL, wr_returned_date_sk::EQUAL) + dynamic filter ([wr_item_sk::EQUAL], await [wr_returned_date_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (d_date_225::EQUAL) + dynamic filter (unestimatable [d_date_225::EQUAL]) scan date_dim final aggregation over (d_date_257) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_257]) partial aggregation over (d_date_257) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (d_week_seq_259::EQUAL) + dynamic filter (await [d_week_seq_259::EQUAL]) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q84.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q84.plan.txt index 4871fa2daadb..13cdc7bb7aab 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q84.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q84.plan.txt @@ -8,18 +8,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_current_hdemo_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_current_cdemo_sk::EQUAL], await [c_current_hdemo_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (cd_demo_sk::EQUAL) + dynamic filter ([cd_demo_sk::EQUAL]) scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (hd_income_band_sk::EQUAL) + dynamic filter (await [hd_income_band_sk::EQUAL]) scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q85.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q85.plan.txt index 2170d63c6768..033a9c711e3c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q85.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q85.plan.txt @@ -15,18 +15,18 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) - dynamic filter (ws_item_sk::EQUAL, ws_order_number::EQUAL, ws_sold_date_sk::EQUAL, ws_web_page_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL, ws_order_number::EQUAL], await [ws_sold_date_sk::EQUAL, ws_web_page_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) - dynamic filter (wr_reason_sk::EQUAL, wr_refunded_addr_sk::EQUAL, wr_refunded_cdemo_sk::EQUAL, wr_returning_cdemo_sk::EQUAL) + dynamic filter ([wr_refunded_addr_sk::EQUAL, wr_refunded_cdemo_sk::EQUAL, wr_returning_cdemo_sk::EQUAL], await [wr_reason_sk::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan web_page local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (cd_education_status::EQUAL, cd_marital_status::EQUAL) + dynamic filter ([cd_education_status::EQUAL, cd_marital_status::EQUAL]) scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q86.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q86.plan.txt index 04a53ea09484..f4595bfe5000 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q86.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q86.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) partial aggregation over (groupid, i_category_gid, i_class_gid) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q87.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q87.plan.txt index 95ea593af612..a1f440be80a2 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q87.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q87.plan.txt @@ -12,7 +12,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ss_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_customer_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -28,7 +28,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [cs_bill_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_bill_customer_sk::EQUAL, cs_sold_date_sk::EQUAL) + dynamic filter ([cs_bill_customer_sk::EQUAL], await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -44,7 +44,7 @@ final aggregation over () join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ws_bill_customer_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_bill_customer_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q88.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q88.plan.txt index 9b9e7d342a33..f42a5b809886 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q88.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q88.plan.txt @@ -12,7 +12,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk::EQUAL, ss_sold_time_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_hdemo_sk::EQUAL, ss_sold_time_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -30,7 +30,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_10::EQUAL, ss_sold_time_sk_6::EQUAL, ss_store_sk_12::EQUAL) + dynamic filter (await [ss_hdemo_sk_10::EQUAL, ss_sold_time_sk_6::EQUAL, ss_store_sk_12::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -48,7 +48,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_86::EQUAL, ss_sold_time_sk_82::EQUAL, ss_store_sk_88::EQUAL) + dynamic filter (await [ss_hdemo_sk_86::EQUAL, ss_sold_time_sk_82::EQUAL, ss_store_sk_88::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -66,7 +66,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_162::EQUAL, ss_sold_time_sk_158::EQUAL, ss_store_sk_164::EQUAL) + dynamic filter (await [ss_hdemo_sk_162::EQUAL, ss_sold_time_sk_158::EQUAL, ss_store_sk_164::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -84,7 +84,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_238::EQUAL, ss_sold_time_sk_234::EQUAL, ss_store_sk_240::EQUAL) + dynamic filter (await [ss_hdemo_sk_238::EQUAL, ss_sold_time_sk_234::EQUAL, ss_store_sk_240::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -102,7 +102,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_314::EQUAL, ss_sold_time_sk_310::EQUAL, ss_store_sk_316::EQUAL) + dynamic filter (await [ss_hdemo_sk_314::EQUAL, ss_sold_time_sk_310::EQUAL, ss_store_sk_316::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -120,7 +120,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_390::EQUAL, ss_sold_time_sk_386::EQUAL, ss_store_sk_392::EQUAL) + dynamic filter (await [ss_hdemo_sk_390::EQUAL, ss_sold_time_sk_386::EQUAL, ss_store_sk_392::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -138,7 +138,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk_466::EQUAL, ss_sold_time_sk_462::EQUAL, ss_store_sk_468::EQUAL) + dynamic filter (await [ss_hdemo_sk_466::EQUAL, ss_sold_time_sk_462::EQUAL, ss_store_sk_468::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q89.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q89.plan.txt index addb32da404d..4a79d7a40c3d 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q89.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q89.plan.txt @@ -11,7 +11,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q90.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q90.plan.txt index b9fc089b9471..f938a3305823 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q90.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q90.plan.txt @@ -6,7 +6,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_ship_hdemo_sk::EQUAL, ws_sold_time_sk::EQUAL, ws_web_page_sk::EQUAL) + dynamic filter (await [ws_ship_hdemo_sk::EQUAL, ws_sold_time_sk::EQUAL, ws_web_page_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -24,7 +24,7 @@ cross join: join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_ship_hdemo_sk_15::EQUAL, ws_sold_time_sk_6::EQUAL, ws_web_page_sk_17::EQUAL) + dynamic filter (await [ws_ship_hdemo_sk_15::EQUAL, ws_sold_time_sk_6::EQUAL, ws_web_page_sk_17::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q91.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q91.plan.txt index 60457033cb17..b6559f6cade8 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q91.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q91.plan.txt @@ -13,7 +13,7 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cr_returning_customer_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cr_call_center_sk::EQUAL, cr_returned_date_sk::EQUAL, cr_returning_customer_sk::EQUAL) + dynamic filter ([cr_returning_customer_sk::EQUAL], await [cr_call_center_sk::EQUAL, cr_returned_date_sk::EQUAL]) scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -23,7 +23,7 @@ remote exchange (GATHER, SINGLE, []) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk]) - dynamic filter (c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL, c_current_hdemo_sk::EQUAL) + dynamic filter ([c_current_addr_sk::EQUAL, c_current_cdemo_sk::EQUAL], await [c_current_hdemo_sk::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q92.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q92.plan.txt index 4d1eef80ffc4..5107d73b3226 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q92.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q92.plan.txt @@ -7,7 +7,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [ws_item_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + dynamic filter ([ws_item_sk::EQUAL], await [ws_sold_date_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -20,7 +20,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [ws_item_sk_6]) partial aggregation over (ws_item_sk_6) join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk_37::EQUAL) + dynamic filter (await [ws_sold_date_sk_37::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q93.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q93.plan.txt index 476bfc0eb4cf..bf8798ce0e5f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q93.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q93.plan.txt @@ -9,11 +9,11 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_item_sk::EQUAL, ss_ticket_number::EQUAL) + dynamic filter ([ss_item_sk::EQUAL, ss_ticket_number::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_reason_sk::EQUAL) + dynamic filter (await [sr_reason_sk::EQUAL]) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q94.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q94.plan.txt index 75621784fed2..893e9f735086 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q94.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q94.plan.txt @@ -13,7 +13,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_order_number::EQUAL, ws_ship_addr_sk::EQUAL, ws_ship_date_sk::EQUAL, ws_web_site_sk::EQUAL) + dynamic filter ([ws_order_number::EQUAL, ws_ship_addr_sk::EQUAL], await [ws_ship_date_sk::EQUAL, ws_web_site_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q95.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q95.plan.txt index e633aad69d8a..5f05af6be610 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q95.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q95.plan.txt @@ -12,7 +12,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ws_order_number::EQUAL, ws_order_number::EQUAL, ws_ship_addr_sk::EQUAL, ws_ship_date_sk::EQUAL, ws_web_site_sk::EQUAL) + dynamic filter ([ws_ship_addr_sk::EQUAL], unestimatable [ws_order_number::EQUAL, ws_order_number::EQUAL], await [ws_ship_date_sk::EQUAL, ws_web_site_sk::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -28,23 +28,23 @@ final aggregation over () partial aggregation over (ws_order_number_22) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ws_order_number_22]) - dynamic filter (ws_order_number_22::EQUAL, ws_order_number_22::EQUAL) + dynamic filter ([ws_order_number_22::EQUAL], unestimatable [ws_order_number_22::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_order_number_58]) - dynamic filter (ws_order_number_58::EQUAL) + dynamic filter (unestimatable [ws_order_number_58::EQUAL]) scan web_sales final aggregation over (wr_order_number) local exchange (GATHER, SINGLE, []) partial aggregation over (wr_order_number) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [wr_order_number]) - dynamic filter (wr_order_number::EQUAL) + dynamic filter (unestimatable [wr_order_number::EQUAL]) scan web_returns local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [ws_order_number_100]) - dynamic filter (ws_order_number_100::EQUAL) + dynamic filter ([ws_order_number_100::EQUAL]) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_order_number_136]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q96.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q96.plan.txt index ca2b95fe53e5..0ed859b07133 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q96.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q96.plan.txt @@ -5,7 +5,7 @@ final aggregation over () join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk::EQUAL, ss_sold_time_sk::EQUAL, ss_store_sk::EQUAL) + dynamic filter (await [ss_hdemo_sk::EQUAL, ss_sold_time_sk::EQUAL, ss_store_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q97.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q97.plan.txt index 6b1600242f13..f8dadad2f1b1 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q97.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q97.plan.txt @@ -8,7 +8,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [ss_customer_sk, ss_item_sk]) partial aggregation over (ss_customer_sk, ss_item_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (ss_sold_date_sk::EQUAL) + dynamic filter (await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -18,7 +18,7 @@ final aggregation over () remote exchange (REPARTITION, HASH, [cs_bill_customer_sk, cs_item_sk]) partial aggregation over (cs_bill_customer_sk, cs_item_sk) join (INNER, REPLICATED, can skip output duplicates): - dynamic filter (cs_sold_date_sk::EQUAL) + dynamic filter (await [cs_sold_date_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q98.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q98.plan.txt index ed3aca1c3c06..c75d3268a654 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q98.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q98.plan.txt @@ -9,7 +9,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + dynamic filter ([ss_item_sk::EQUAL], await [ss_sold_date_sk::EQUAL]) scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q99.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q99.plan.txt index 42edd5585a9f..1bc73e19468c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q99.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpcds/iceberg/partitioned/q99.plan.txt @@ -10,7 +10,7 @@ local exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (cs_call_center_sk::EQUAL, cs_ship_date_sk::EQUAL, cs_ship_mode_sk::EQUAL, cs_warehouse_sk::EQUAL) + dynamic filter (await [cs_call_center_sk::EQUAL, cs_ship_date_sk::EQUAL, cs_ship_mode_sk::EQUAL, cs_warehouse_sk::EQUAL]) scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q02.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q02.plan.txt index f874dbf363a9..f206dfabc987 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q02.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q02.plan.txt @@ -10,19 +10,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_5]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey_4]) - dynamic filter (suppkey_5::EQUAL) + dynamic filter ([suppkey_5::EQUAL]) scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (regionkey::EQUAL) + dynamic filter (await [regionkey::EQUAL]) scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -35,15 +35,15 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey_20]) - dynamic filter (suppkey_20::EQUAL) + dynamic filter ([suppkey_20::EQUAL]) scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_26]) - dynamic filter (nationkey_29::EQUAL) + dynamic filter (await [nationkey_29::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (regionkey_37::EQUAL) + dynamic filter (await [regionkey_37::EQUAL]) scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q03.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q03.plan.txt index 315c89c3c97c..e92fb3b23b69 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q03.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q03.plan.txt @@ -9,7 +9,7 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [custkey_0]) - dynamic filter (custkey_0::EQUAL, orderkey::EQUAL) + dynamic filter ([custkey_0::EQUAL, orderkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q04.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q04.plan.txt index 1c39d860814b..d09ffc2bbd24 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q04.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q04.plan.txt @@ -10,7 +10,7 @@ remote exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_0]) partial aggregation over (orderkey_0) - dynamic filter (orderkey_0::EQUAL) + dynamic filter ([orderkey_0::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q05.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q05.plan.txt index bbc3f5312054..052c4cefcc5f 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q05.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q05.plan.txt @@ -13,23 +13,23 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [custkey_0]) - dynamic filter (custkey_0::EQUAL, orderkey::EQUAL) + dynamic filter ([custkey_0::EQUAL, orderkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey]) - dynamic filter (nationkey::EQUAL, nationkey::EQUAL) + dynamic filter ([nationkey::EQUAL], await [nationkey::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_4]) - dynamic filter (suppkey::EQUAL) + dynamic filter ([suppkey::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [nationkey_11, suppkey_8]) - dynamic filter (nationkey_11::EQUAL) + dynamic filter (await [nationkey_11::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (regionkey::EQUAL) + dynamic filter (await [regionkey::EQUAL]) scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q07.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q07.plan.txt index 636ec16a1d0c..66f2748a4e91 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q07.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q07.plan.txt @@ -13,19 +13,19 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey_0]) - dynamic filter (orderkey::EQUAL, suppkey_0::EQUAL) + dynamic filter ([orderkey::EQUAL, suppkey_0::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_4]) - dynamic filter (custkey::EQUAL) + dynamic filter ([custkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey_8]) - dynamic filter (nationkey_11::EQUAL) + dynamic filter (await [nationkey_11::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q08.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q08.plan.txt index f2483f068666..4e042cadea73 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q08.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q08.plan.txt @@ -16,26 +16,26 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_5]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey_4]) - dynamic filter (orderkey::EQUAL, partkey_4::EQUAL, suppkey_5::EQUAL) + dynamic filter ([orderkey::EQUAL, partkey_4::EQUAL, suppkey_5::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey]) scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_9]) - dynamic filter (custkey::EQUAL) + dynamic filter ([custkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey_13]) - dynamic filter (nationkey_16::EQUAL) + dynamic filter (await [nationkey_16::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (regionkey::EQUAL) + dynamic filter (await [regionkey::EQUAL]) scan nation local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q09.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q09.plan.txt index 5a64c32bc71e..11209a11ddbe 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q09.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q09.plan.txt @@ -13,15 +13,15 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_5]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey_4]) - dynamic filter (orderkey::EQUAL, partkey_4::EQUAL, partkey_4::EQUAL, suppkey_5::EQUAL, suppkey_5::EQUAL) + dynamic filter ([orderkey::EQUAL, partkey_4::EQUAL, partkey_4::EQUAL, suppkey_5::EQUAL, suppkey_5::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (nationkey::EQUAL, suppkey::EQUAL) + dynamic filter ([suppkey::EQUAL], await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_10]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q10.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q10.plan.txt index 5fac1b400cb1..468bcfe5a285 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q10.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q10.plan.txt @@ -10,15 +10,15 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [orderkey]) - dynamic filter (orderkey::EQUAL) + dynamic filter ([orderkey::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_0]) - dynamic filter (custkey::EQUAL) + dynamic filter ([custkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey_4]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q11.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q11.plan.txt index d2fcfb56f509..97d534608b6d 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q11.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q11.plan.txt @@ -9,11 +9,11 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (suppkey::EQUAL) + dynamic filter ([suppkey::EQUAL]) scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_0]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -27,11 +27,11 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey_10]) - dynamic filter (suppkey_10::EQUAL) + dynamic filter ([suppkey_10::EQUAL]) scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey_16]) - dynamic filter (nationkey_19::EQUAL) + dynamic filter (await [nationkey_19::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q12.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q12.plan.txt index 71e114b465a2..9a1834a995ed 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q12.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q12.plan.txt @@ -7,7 +7,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (shipmode) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [orderkey]) - dynamic filter (orderkey::EQUAL) + dynamic filter ([orderkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_0]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q13.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q13.plan.txt index a99fd2f5e1e4..47f9da249d8c 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q13.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q13.plan.txt @@ -10,7 +10,7 @@ remote exchange (GATHER, SINGLE, []) partial aggregation over (custkey) join (RIGHT, PARTITIONED): remote exchange (REPARTITION, HASH, [custkey_0]) - dynamic filter (custkey_0::EQUAL) + dynamic filter ([custkey_0::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q14.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q14.plan.txt index 28a234d25ffb..9225a8fe6ead 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q14.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q14.plan.txt @@ -4,7 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey_0]) - dynamic filter (partkey_0::EQUAL) + dynamic filter ([partkey_0::EQUAL]) scan part local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q15.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q15.plan.txt index bf2eedeaf465..fa9f01e4b463 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q15.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q15.plan.txt @@ -4,7 +4,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, REPLICATED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (suppkey::EQUAL) + dynamic filter ([suppkey::EQUAL]) scan supplier final aggregation over (suppkey_0) local exchange (GATHER, SINGLE, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q16.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q16.plan.txt index de2a83ec7f99..b77ebb628876 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q16.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q16.plan.txt @@ -12,7 +12,7 @@ remote exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan partsupp local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey_0]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q17.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q17.plan.txt index 859578395e19..e89cae69b61b 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q17.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q17.plan.txt @@ -6,7 +6,7 @@ final aggregation over () join (LEFT, PARTITIONED): join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey_0]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q18.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q18.plan.txt index 1e4e3258c636..9a79bb6c1fe7 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q18.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q18.plan.txt @@ -10,14 +10,14 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [custkey_0]) - dynamic filter (custkey_0::EQUAL, orderkey::EQUAL, orderkey::EQUAL) + dynamic filter ([custkey_0::EQUAL, orderkey::EQUAL, orderkey::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [custkey]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_4]) - dynamic filter (orderkey_4::EQUAL) + dynamic filter ([orderkey_4::EQUAL]) scan lineitem final aggregation over (orderkey_8) local exchange (GATHER, SINGLE, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q19.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q19.plan.txt index 3da7d08411d3..b27df635ecc0 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q19.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q19.plan.txt @@ -4,7 +4,7 @@ final aggregation over () partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [partkey_0]) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q20.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q20.plan.txt index c2068d10857c..447017d40c53 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q20.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q20.plan.txt @@ -4,7 +4,7 @@ remote exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey]) join (INNER, REPLICATED): - dynamic filter (nationkey::EQUAL, suppkey::EQUAL) + dynamic filter ([suppkey::EQUAL], await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) @@ -17,7 +17,7 @@ remote exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED, can skip output duplicates): join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [partkey]) - dynamic filter (partkey::EQUAL) + dynamic filter ([partkey::EQUAL]) scan partsupp final aggregation over (partkey_11) local exchange (GATHER, SINGLE, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q21.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q21.plan.txt index af09110ef79d..79cc0e18add7 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q21.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q21.plan.txt @@ -17,15 +17,15 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey]) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [suppkey_0]) - dynamic filter (orderkey::EQUAL, orderkey::EQUAL, suppkey_0::EQUAL) + dynamic filter ([orderkey::EQUAL, orderkey::EQUAL, suppkey_0::EQUAL]) scan lineitem local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [suppkey]) - dynamic filter (nationkey::EQUAL) + dynamic filter (await [nationkey::EQUAL]) scan supplier local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [orderkey_4]) - dynamic filter (orderkey_4::EQUAL) + dynamic filter ([orderkey_4::EQUAL]) scan orders local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) diff --git a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q22.plan.txt b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q22.plan.txt index 3434e8342e1a..37c27513df08 100644 --- a/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q22.plan.txt +++ b/testing/trino-tests/src/test/resources/sql/trino/tpch/iceberg/unpartitioned/q22.plan.txt @@ -8,7 +8,7 @@ remote exchange (GATHER, SINGLE, []) join (LEFT, PARTITIONED): remote exchange (REPARTITION, HASH, [custkey]) cross join: - dynamic filter (acctbal::GREATER_THAN) + dynamic filter (await [acctbal::GREATER_THAN]) scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, [])