From af08170336a68aed531d5c5e9082055d0188fa44 Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Mon, 28 Aug 2023 17:24:32 +0200 Subject: [PATCH 1/2] Add AggregationNode#isInputReducingAggregation --- .../sql/planner/plan/AggregationNode.java | 39 ++++++++++++++++++- 1 file changed, 37 insertions(+), 2 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/plan/AggregationNode.java b/core/trino-main/src/main/java/io/trino/sql/planner/plan/AggregationNode.java index da2b6e76fc3d..1c797eb0faec 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/plan/AggregationNode.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/plan/AggregationNode.java @@ -53,6 +53,11 @@ public class AggregationNode private final Optional hashSymbol; private final Optional groupIdSymbol; private final List outputs; + /** + * Indicates whether it is beneficial (e.g. reduces remote exchange input) to retain this aggregation + * as an auxiliary step when making a decision to push down partial aggregation more aggressively. + */ + private final Optional isInputReducingAggregation; public static AggregationNode singleAggregation( PlanNodeId id, @@ -63,6 +68,19 @@ public static AggregationNode singleAggregation( return new AggregationNode(id, source, aggregations, groupingSets, ImmutableList.of(), SINGLE, Optional.empty(), Optional.empty()); } + public AggregationNode( + PlanNodeId id, + PlanNode source, + Map aggregations, + GroupingSetDescriptor groupingSets, + List preGroupedSymbols, + Step step, + Optional hashSymbol, + Optional groupIdSymbol) + { + this(id, source, aggregations, groupingSets, preGroupedSymbols, step, hashSymbol, groupIdSymbol, Optional.empty()); + } + @JsonCreator public AggregationNode( @JsonProperty("id") PlanNodeId id, @@ -72,7 +90,8 @@ public AggregationNode( @JsonProperty("preGroupedSymbols") List preGroupedSymbols, @JsonProperty("step") Step step, @JsonProperty("hashSymbol") Optional hashSymbol, - @JsonProperty("groupIdSymbol") Optional groupIdSymbol) + @JsonProperty("groupIdSymbol") Optional groupIdSymbol, + @JsonProperty("isInputReducingAggregation") Optional isInputReducingAggregation) { super(id); @@ -104,6 +123,7 @@ public AggregationNode( outputs.addAll(aggregations.keySet()); this.outputs = outputs.build(); + this.isInputReducingAggregation = requireNonNull(isInputReducingAggregation, "exchangeInputAggregation is null"); } public List getGroupingKeys() @@ -207,6 +227,12 @@ public Optional getGroupIdSymbol() return groupIdSymbol; } + @JsonProperty("isInputReducingAggregation") + public boolean isInputReducingAggregation() + { + return isInputReducingAggregation.orElse(false); + } + public boolean hasOrderings() { return aggregations.values().stream() @@ -513,6 +539,7 @@ public static class Builder private Step step; private Optional hashSymbol; private Optional groupIdSymbol; + private Optional isInputReducingAggregation; public Builder(AggregationNode node) { @@ -525,6 +552,7 @@ public Builder(AggregationNode node) this.step = node.getStep(); this.hashSymbol = node.getHashSymbol(); this.groupIdSymbol = node.getGroupIdSymbol(); + this.isInputReducingAggregation = node.isInputReducingAggregation; } public Builder setId(PlanNodeId id) @@ -575,6 +603,12 @@ public Builder setGroupIdSymbol(Optional groupIdSymbol) return this; } + public Builder setIsInputReducingAggregation(boolean isInputReducingAggregation) + { + this.isInputReducingAggregation = Optional.of(isInputReducingAggregation); + return this; + } + public AggregationNode build() { return new AggregationNode( @@ -585,7 +619,8 @@ public AggregationNode build() preGroupedSymbols, step, hashSymbol, - groupIdSymbol); + groupIdSymbol, + isInputReducingAggregation); } } } From b9f03740651c0b8636c3fa5ca44e6367d07023ac Mon Sep 17 00:00:00 2001 From: Karol Sobczak Date: Tue, 14 Mar 2023 17:15:17 +0100 Subject: [PATCH 2/2] Enable push partial aggregation though join Make push partial aggregation CBO based. Enable it for cases where pushed aggregation has same grouping keys. Additionally, for queries like select sum(sales) from fact, date_dim where fact.date_id = date_dim.date_id group by date_dim.year partial aggregation on date_dim.year can be pushed below join with grouping key of "date_id", which can greatly reduce number of rows before join operator. --- .../io/trino/sql/planner/OptimizerConfig.java | 2 +- .../io/trino/sql/planner/PlanOptimizers.java | 12 +- ...PushPartialAggregationThroughExchange.java | 3 + .../PushPartialAggregationThroughJoin.java | 236 +++++++++++-- .../io/trino/cost/TestOptimizerConfig.java | 6 +- ...TestPushPartialAggregationThroughJoin.java | 329 +++++++++++++++++- .../iterative/rule/test/PlanBuilder.java | 40 ++- .../tpcds/iceberg/partitioned/q02.plan.txt | 24 +- .../tpcds/iceberg/partitioned/q03.plan.txt | 7 +- .../tpcds/iceberg/partitioned/q04.plan.txt | 102 +++--- .../tpcds/iceberg/partitioned/q05.plan.txt | 75 ++-- .../tpcds/iceberg/partitioned/q07.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q11.plan.txt | 68 ++-- .../tpcds/iceberg/partitioned/q12.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q14.plan.txt | 180 +++++----- .../tpcds/iceberg/partitioned/q20.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q21.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q23.plan.txt | 68 ++-- .../tpcds/iceberg/partitioned/q26.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q31.plan.txt | 96 ++--- .../tpcds/iceberg/partitioned/q33.plan.txt | 51 +-- .../tpcds/iceberg/partitioned/q38.plan.txt | 48 +-- .../tpcds/iceberg/partitioned/q39.plan.txt | 32 +- .../tpcds/iceberg/partitioned/q40.plan.txt | 25 +- .../tpcds/iceberg/partitioned/q42.plan.txt | 7 +- .../tpcds/iceberg/partitioned/q43.plan.txt | 7 +- .../tpcds/iceberg/partitioned/q46.plan.txt | 12 +- .../tpcds/iceberg/partitioned/q47.plan.txt | 21 +- .../tpcds/iceberg/partitioned/q50.plan.txt | 33 +- .../tpcds/iceberg/partitioned/q51.plan.txt | 14 +- .../tpcds/iceberg/partitioned/q52.plan.txt | 7 +- .../tpcds/iceberg/partitioned/q55.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q56.plan.txt | 34 +- .../tpcds/iceberg/partitioned/q57.plan.txt | 21 +- .../tpcds/iceberg/partitioned/q58.plan.txt | 115 +++--- .../tpcds/iceberg/partitioned/q59.plan.txt | 14 +- .../tpcds/iceberg/partitioned/q60.plan.txt | 51 +-- .../tpcds/iceberg/partitioned/q62.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q64.plan.txt | 272 ++++++++------- .../tpcds/iceberg/partitioned/q66.plan.txt | 38 +- .../tpcds/iceberg/partitioned/q68.plan.txt | 19 +- .../tpcds/iceberg/partitioned/q70.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q71.plan.txt | 41 ++- .../tpcds/iceberg/partitioned/q74.plan.txt | 64 ++-- .../tpcds/iceberg/partitioned/q76.plan.txt | 33 +- .../tpcds/iceberg/partitioned/q77.plan.txt | 40 +-- .../tpcds/iceberg/partitioned/q78.plan.txt | 53 +-- .../tpcds/iceberg/partitioned/q79.plan.txt | 17 +- .../tpcds/iceberg/partitioned/q80.plan.txt | 87 ++--- .../tpcds/iceberg/partitioned/q81.plan.txt | 30 +- .../tpcds/iceberg/partitioned/q83.plan.txt | 149 ++++---- .../tpcds/iceberg/partitioned/q87.plan.txt | 48 +-- .../tpcds/iceberg/partitioned/q89.plan.txt | 7 +- .../tpcds/iceberg/partitioned/q91.plan.txt | 45 +-- .../tpcds/iceberg/partitioned/q98.plan.txt | 15 +- .../tpcds/iceberg/partitioned/q99.plan.txt | 15 +- .../tpch/iceberg/unpartitioned/q03.plan.txt | 22 +- .../tpch/iceberg/unpartitioned/q18.plan.txt | 38 +- .../tpch/iceberg/unpartitioned/q21.plan.txt | 7 +- 59 files changed, 1770 insertions(+), 1138 deletions(-) diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java b/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java index 54f0bceedb3d..7678c2a97779 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/OptimizerConfig.java @@ -75,7 +75,7 @@ public class OptimizerConfig private boolean preferPartialAggregation = true; private boolean pushAggregationThroughOuterJoin = true; private boolean enableIntermediateAggregations; - private boolean pushPartialAggregationThroughJoin; + private boolean pushPartialAggregationThroughJoin = true; private boolean preAggregateCaseAggregationsEnabled = true; private boolean enableForcedExchangeBelowGroupId = true; private boolean optimizeTopNRanking = true; 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 3682023f88a5..37c06649c3be 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 @@ -1000,11 +1000,13 @@ public PlanOptimizers( ruleStats, statsCalculator, costCalculator, - ImmutableSet.of( - new PushPartialAggregationThroughJoin(), - new PushPartialAggregationThroughExchange(plannerContext), - new PruneJoinColumns(), - new PruneJoinChildrenColumns()))); + ImmutableSet.>builder() + .addAll(new PushPartialAggregationThroughJoin().rules()) + .add(new PushPartialAggregationThroughExchange(plannerContext), + new PruneJoinColumns(), + new PruneJoinChildrenColumns(), + new RemoveRedundantIdentityProjections()) + .build())); builder.add(new IterativeOptimizer( plannerContext, ruleStats, diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java index 6ebc4543493e..7df67f5bcfea 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughExchange.java @@ -157,6 +157,9 @@ private PlanNode pushPartial(AggregationNode aggregation, ExchangeNode exchange, SymbolMapper symbolMapper = mappingsBuilder.build(); AggregationNode mappedPartial = symbolMapper.map(aggregation, source, context.getIdAllocator().getNextId()); + mappedPartial = AggregationNode.builderFrom(mappedPartial) + .setIsInputReducingAggregation(true) + .build(); Assignments.Builder assignments = Assignments.builder(); diff --git a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java index 4287a63317e4..7eb830c3da36 100644 --- a/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java +++ b/core/trino-main/src/main/java/io/trino/sql/planner/iterative/rule/PushPartialAggregationThroughJoin.java @@ -17,43 +17,50 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Streams; import io.trino.Session; -import io.trino.matching.Capture; +import io.trino.cost.PlanNodeStatsEstimate; import io.trino.matching.Captures; import io.trino.matching.Pattern; +import io.trino.metadata.ResolvedFunction; +import io.trino.sql.ir.Expression; +import io.trino.sql.ir.Lambda; import io.trino.sql.planner.Symbol; import io.trino.sql.planner.SymbolsExtractor; import io.trino.sql.planner.iterative.Rule; +import io.trino.sql.planner.iterative.Rule.Context; +import io.trino.sql.planner.iterative.Rule.Result; import io.trino.sql.planner.plan.AggregationNode; import io.trino.sql.planner.plan.AggregationNode.Aggregation; import io.trino.sql.planner.plan.JoinNode; import io.trino.sql.planner.plan.JoinType; import io.trino.sql.planner.plan.PlanNode; +import io.trino.sql.planner.plan.ProjectNode; +import org.assertj.core.util.VisibleForTesting; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import static com.google.common.collect.ImmutableList.toImmutableList; import static com.google.common.collect.ImmutableSet.toImmutableSet; import static com.google.common.collect.Sets.intersection; import static io.trino.SystemSessionProperties.isPushPartialAggregationThroughJoin; +import static io.trino.sql.planner.iterative.rule.PushProjectionThroughJoin.pushProjectionThroughJoin; import static io.trino.sql.planner.iterative.rule.Util.restrictOutputs; +import static io.trino.sql.planner.plan.AggregationNode.Step.INTERMEDIATE; import static io.trino.sql.planner.plan.AggregationNode.Step.PARTIAL; import static io.trino.sql.planner.plan.AggregationNode.singleGroupingSet; import static io.trino.sql.planner.plan.Patterns.aggregation; import static io.trino.sql.planner.plan.Patterns.join; +import static io.trino.sql.planner.plan.Patterns.project; import static io.trino.sql.planner.plan.Patterns.source; +import static java.lang.Double.isNaN; public class PushPartialAggregationThroughJoin - implements Rule { - private static final Capture JOIN_NODE = Capture.newCapture(); - - private static final Pattern PATTERN = aggregation() - .matching(PushPartialAggregationThroughJoin::isSupportedAggregationNode) - .with(source().matching(join().capturedAs(JOIN_NODE))); - private static boolean isSupportedAggregationNode(AggregationNode aggregationNode) { // Don't split streaming aggregations @@ -68,33 +75,95 @@ private static boolean isSupportedAggregationNode(AggregationNode aggregationNod return aggregationNode.getStep() == PARTIAL && aggregationNode.getGroupingSetCount() == 1; } - @Override - public Pattern getPattern() + public Iterable> rules() + { + return ImmutableList.of( + pushPartialAggregationThroughJoinWithoutProjection(), + pushPartialAggregationThroughJoinWithProjection()); + } + + @VisibleForTesting + Rule pushPartialAggregationThroughJoinWithoutProjection() + { + return new PushPartialAggregationThroughJoinWithoutProjection(); + } + + @VisibleForTesting + Rule pushPartialAggregationThroughJoinWithProjection() + { + return new PushPartialAggregationThroughJoinWithProjection(); + } + + private class PushPartialAggregationThroughJoinWithoutProjection + implements Rule { - return PATTERN; + private static final Pattern PATTERN_WITHOUT_PROJECTION = aggregation() + .matching(PushPartialAggregationThroughJoin::isSupportedAggregationNode) + .with(source().matching(join())); + + @Override + public Pattern getPattern() + { + return PATTERN_WITHOUT_PROJECTION; + } + + @Override + public boolean isEnabled(Session session) + { + return isPushPartialAggregationThroughJoin(session); + } + + @Override + public Result apply(AggregationNode node, Captures captures, Context context) + { + return applyPushdown(node, context); + } } - @Override - public boolean isEnabled(Session session) + private class PushPartialAggregationThroughJoinWithProjection + implements Rule { - return isPushPartialAggregationThroughJoin(session); + private static final Pattern PATTERN_WITH_PROJECTION = aggregation() + .matching(PushPartialAggregationThroughJoin::isSupportedAggregationNode) + .with(source().matching(project().with(source().matching(join())))); + + @Override + public Pattern getPattern() + { + return PATTERN_WITH_PROJECTION; + } + + @Override + public boolean isEnabled(Session session) + { + return isPushPartialAggregationThroughJoin(session); + } + + @Override + public Result apply(AggregationNode node, Captures captures, Context context) + { + ProjectNode projectNode = (ProjectNode) context.getLookup().resolve(node.getSource()); + Optional joinNodeOptional = pushProjectionThroughJoin(projectNode, context.getLookup(), context.getIdAllocator()); + if (joinNodeOptional.isEmpty()) { + return Result.empty(); + } + return applyPushdown((AggregationNode) node.replaceChildren(ImmutableList.of(joinNodeOptional.get())), context); + } } - @Override - public Result apply(AggregationNode aggregationNode, Captures captures, Context context) + private Result applyPushdown(AggregationNode aggregationNode, Context context) { - JoinNode joinNode = captures.get(JOIN_NODE); + JoinNode joinNode = (JoinNode) context.getLookup().resolve(aggregationNode.getSource()); if (joinNode.getType() != JoinType.INNER) { return Result.empty(); } - // TODO: leave partial aggregation above Join? if (allAggregationsOn(aggregationNode.getAggregations(), joinNode.getLeft().getOutputSymbols())) { - return Result.ofPlanNode(pushPartialToLeftChild(aggregationNode, joinNode, context)); + return pushPartialToLeftChild(aggregationNode, joinNode, context).map(Result::ofPlanNode).orElse(Result.empty()); } if (allAggregationsOn(aggregationNode.getAggregations(), joinNode.getRight().getOutputSymbols())) { - return Result.ofPlanNode(pushPartialToRightChild(aggregationNode, joinNode, context)); + return pushPartialToRightChild(aggregationNode, joinNode, context).map(Result::ofPlanNode).orElse(Result.empty()); } return Result.empty(); @@ -109,20 +178,75 @@ private static boolean allAggregationsOn(Map aggregations, return symbols.containsAll(inputs); } - private PlanNode pushPartialToLeftChild(AggregationNode node, JoinNode child, Context context) + private Optional pushPartialToLeftChild(AggregationNode node, JoinNode child, Context context) + { + return getPushedAggregation(node, child, child.getLeft(), context) + .map(pushedAggregation -> replaceJoin(node, pushedAggregation, child, pushedAggregation, child.getRight(), context)); + } + + private Optional pushPartialToRightChild(AggregationNode node, JoinNode child, Context context) { - Set joinLeftChildSymbols = ImmutableSet.copyOf(child.getLeft().getOutputSymbols()); - List groupingSet = getPushedDownGroupingSet(node, joinLeftChildSymbols, intersection(getJoinRequiredSymbols(child), joinLeftChildSymbols)); - AggregationNode pushedAggregation = replaceAggregationSource(node, child.getLeft(), groupingSet); - return pushPartialToJoin(node, child, pushedAggregation, child.getRight(), context); + return getPushedAggregation(node, child, child.getRight(), context) + .map(pushedAggregation -> replaceJoin(node, pushedAggregation, child, child.getLeft(), pushedAggregation, context)); } - private PlanNode pushPartialToRightChild(AggregationNode node, JoinNode child, Context context) + private Optional getPushedAggregation(AggregationNode node, JoinNode child, PlanNode joinSource, Context context) { - Set joinRightChildSymbols = ImmutableSet.copyOf(child.getRight().getOutputSymbols()); - List groupingSet = getPushedDownGroupingSet(node, joinRightChildSymbols, intersection(getJoinRequiredSymbols(child), joinRightChildSymbols)); - AggregationNode pushedAggregation = replaceAggregationSource(node, child.getRight(), groupingSet); - return pushPartialToJoin(node, child, child.getLeft(), pushedAggregation, context); + Set joinSourceSymbols = ImmutableSet.copyOf(joinSource.getOutputSymbols()); + List groupingSet = getPushedDownGroupingSet(node, joinSourceSymbols, intersection(getJoinRequiredSymbols(child), joinSourceSymbols)); + AggregationNode pushedAggregation = replaceAggregationSource(node, joinSource, groupingSet); + if (skipPartialAggregationPushdown(child, node, pushedAggregation, context)) { + return Optional.empty(); + } + return Optional.of(pushedAggregation); + } + + private boolean skipPartialAggregationPushdown( + JoinNode join, + AggregationNode originalAggregation, + AggregationNode pushedAggregation, + Context context) + { + // Only push aggregation down if pushed aggregation consumes similar number of input rows (e.g. join is not expanding). + // Otherwise, there is a possibility that aggregation above join could be more efficient in reducing total number of rows. + PlanNodeStatsEstimate sourceStats = context.getStatsProvider().getStats(pushedAggregation.getSource()); + double sourceRowCount = sourceStats.getOutputRowCount(); + double joinRowCount = context.getStatsProvider().getStats(join).getOutputRowCount(); + // Pushing aggregation through filtering join could mean more work for partial aggregation. However, + // we allow pushing partial aggregations through filtering join because: + // 1. dynamic filtering should filter unmatched rows at source + // 2. partial aggregation will adaptively switch off when it's not reducing input rows + // 3. join operator is not particularly efficient at filtering rows + if (isNaN(sourceRowCount) || isNaN(joinRowCount) || joinRowCount > 1.1 * sourceRowCount) { + return true; + } + + // Only push aggregation down if pushed grouping set is of same size or smaller. This is because + // we want pushdown to happen for star schema queries like: + // + // select sum(sales) from fact, date_dim where fact.date_id = date_dim.date_id group by date_dim.year + // + // In such case partial aggregation on date_dim.year can be pushed + // below join with grouping key of "date_id". This can greatly reduce number + // of rows before join operator. + if (ImmutableSet.copyOf(originalAggregation.getGroupingKeys()).size() < ImmutableSet.copyOf(pushedAggregation.getGroupingKeys()).size()) { + return true; + } + + // Do not push aggregation down if any pushed grouping symbol has NDV that has the + // same order of magnitude as number of source rows (because then partial aggregation is ineffective). + // Ideally we should use estimated aggregation row count. However, we assume lack of correlation + // between group by columns in stats calculations. Therefore, if we used estimated aggregation row count + // then we would miss good improvement opportunities. Even if partial aggregation is pushed down incorrectly, + // it should be adaptively turned off, hence potential performance penalty is not that significant. + for (Symbol symbol : pushedAggregation.getGroupingKeys()) { + double ndv = sourceStats.getSymbolStatistics(symbol).getDistinctValuesCount(); + if (isNaN(ndv) || ndv * 2 > sourceRowCount) { + return true; + } + } + + return false; } private Set getJoinRequiredSymbols(JoinNode node) @@ -163,11 +287,14 @@ private AggregationNode replaceAggregationSource( .setSource(source) .setGroupingSets(singleGroupingSet(groupingKeys)) .setPreGroupedSymbols(ImmutableList.of()) + // aggregation below join might not be as effective in reducing rows before exchange + .setIsInputReducingAggregation(false) .build(); } - private PlanNode pushPartialToJoin( + private PlanNode replaceJoin( AggregationNode aggregation, + AggregationNode pushedAggregation, JoinNode child, PlanNode leftChild, PlanNode rightChild, @@ -189,6 +316,51 @@ private PlanNode pushPartialToJoin( child.isSpillable(), child.getDynamicFilters(), child.getReorderJoinStatsAndCost()); - return restrictOutputs(context.getIdAllocator(), joinNode, ImmutableSet.copyOf(aggregation.getOutputSymbols())).orElse(joinNode); + PlanNode result = restrictOutputs(context.getIdAllocator(), joinNode, ImmutableSet.copyOf(aggregation.getOutputSymbols())).orElse(joinNode); + // Keep intermediate aggregation below remote exchange to reduce network traffic. + // Intermediate aggregation can be skipped if pushed aggregation has subset of grouping + // symbols as join is not expanding. + if (aggregation.isInputReducingAggregation() && !ImmutableSet.copyOf(aggregation.getGroupingKeys()).containsAll(pushedAggregation.getGroupingKeys())) { + result = toIntermediateAggregation(aggregation, result, context); + } + return result; + } + + private PlanNode toIntermediateAggregation(AggregationNode partialAggregation, PlanNode source, Context context) + { + Map intermediateAggregation = new HashMap<>(); + for (Map.Entry entry : partialAggregation.getAggregations().entrySet()) { + AggregationNode.Aggregation aggregation = entry.getValue(); + ResolvedFunction resolvedFunction = aggregation.getResolvedFunction(); + + // rewrite partial aggregation in terms of intermediate function + intermediateAggregation.put( + entry.getKey(), + new AggregationNode.Aggregation( + resolvedFunction, + ImmutableList.builder() + .add(entry.getKey().toSymbolReference()) + .addAll(aggregation.getArguments().stream() + .filter(Lambda.class::isInstance) + .collect(toImmutableList())) + .build(), + false, + Optional.empty(), + Optional.empty(), + Optional.empty())); + } + + return new AggregationNode( + context.getIdAllocator().getNextId(), + source, + intermediateAggregation, + partialAggregation.getGroupingSets(), + // preGroupedSymbols reflect properties of the input. Splitting the aggregation and pushing partial aggregation + // through the join may or may not preserve these properties. Hence, it is safest to drop preGroupedSymbols here. + ImmutableList.of(), + INTERMEDIATE, + // hash symbol is not supported by this rule + Optional.empty(), + partialAggregation.getGroupIdSymbol()); } } diff --git a/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java b/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java index a461a420f454..f9308c50794f 100644 --- a/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java +++ b/core/trino-main/src/test/java/io/trino/cost/TestOptimizerConfig.java @@ -67,7 +67,7 @@ public void testDefaults() .setEnableForcedExchangeBelowGroupId(true) .setEnableIntermediateAggregations(false) .setPushAggregationThroughOuterJoin(true) - .setPushPartialAggregationThroughJoin(false) + .setPushPartialAggregationThroughJoin(true) .setPreAggregateCaseAggregationsEnabled(true) .setDistinctAggregationsStrategy(null) .setPreferPartialAggregation(true) @@ -126,7 +126,7 @@ public void testExplicitPropertyMappings() .put("optimizer.push-table-write-through-union", "false") .put("optimizer.dictionary-aggregation", "true") .put("optimizer.push-aggregation-through-outer-join", "false") - .put("optimizer.push-partial-aggregation-through-join", "true") + .put("optimizer.push-partial-aggregation-through-join", "false") .put("optimizer.pre-aggregate-case-aggregations.enabled", "false") .put("optimizer.enable-intermediate-aggregations", "true") .put("optimizer.force-single-node-output", "true") @@ -181,7 +181,7 @@ public void testExplicitPropertyMappings() .setPushTableWriteThroughUnion(false) .setDictionaryAggregation(true) .setPushAggregationThroughOuterJoin(false) - .setPushPartialAggregationThroughJoin(true) + .setPushPartialAggregationThroughJoin(false) .setPreAggregateCaseAggregationsEnabled(false) .setEnableIntermediateAggregations(true) .setDistinctAggregationsStrategy(MARK_DISTINCT) diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java index 568762e3e62b..375eb1f4ab4d 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/TestPushPartialAggregationThroughJoin.java @@ -15,17 +15,24 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.trino.cost.PlanNodeStatsEstimate; +import io.trino.cost.SymbolStatsEstimate; +import io.trino.metadata.ResolvedFunction; +import io.trino.metadata.TestingFunctionResolution; +import io.trino.spi.function.OperatorType; +import io.trino.sql.ir.Call; import io.trino.sql.ir.Comparison; import io.trino.sql.ir.Reference; +import io.trino.sql.planner.Symbol; import io.trino.sql.planner.assertions.PlanMatchPattern; import io.trino.sql.planner.iterative.rule.test.BaseRuleTest; -import io.trino.sql.planner.iterative.rule.test.PlanBuilder; +import io.trino.sql.planner.plan.Assignments; import io.trino.sql.planner.plan.JoinNode.EquiJoinClause; +import io.trino.sql.planner.plan.PlanNodeId; import org.junit.jupiter.api.Test; import java.util.Optional; -import static io.trino.SystemSessionProperties.PUSH_PARTIAL_AGGREGATION_THROUGH_JOIN; import static io.trino.spi.type.BigintType.BIGINT; import static io.trino.spi.type.DoubleType.DOUBLE; import static io.trino.sql.ir.Comparison.Operator.LESS_THAN_OR_EQUAL; @@ -35,47 +42,333 @@ import static io.trino.sql.planner.assertions.PlanMatchPattern.project; import static io.trino.sql.planner.assertions.PlanMatchPattern.singleGroupingSet; import static io.trino.sql.planner.assertions.PlanMatchPattern.values; +import static io.trino.sql.planner.iterative.rule.test.PlanBuilder.aggregation; +import static io.trino.sql.planner.plan.AggregationNode.Step.INTERMEDIATE; import static io.trino.sql.planner.plan.AggregationNode.Step.PARTIAL; import static io.trino.sql.planner.plan.JoinType.INNER; +import static java.lang.Double.NaN; public class TestPushPartialAggregationThroughJoin extends BaseRuleTest { + private static final PlanNodeId JOIN_ID = new PlanNodeId("join_id"); + private static final PlanNodeId CHILD_ID = new PlanNodeId("child_id"); + + private static final TestingFunctionResolution FUNCTIONS = new TestingFunctionResolution(); + private static final ResolvedFunction ADD_BIGINT = FUNCTIONS.resolveOperator(OperatorType.ADD, ImmutableList.of(BIGINT, BIGINT)); + @Test - public void testPushesPartialAggregationThroughJoin() + public void testPushesPartialAggregationThroughJoinToLeftChildWithoutProjection() { - tester().assertThat(new PushPartialAggregationThroughJoin()) - .setSystemProperty(PUSH_PARTIAL_AGGREGATION_THROUGH_JOIN, "true") + // push to left child + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) .on(p -> p.aggregation(ab -> ab .source( p.join( INNER, - p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR"), p.symbol("LEFT_HASH")), - p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI"), p.symbol("RIGHT_GROUP_BY"), p.symbol("RIGHT_HASH")), + p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")), ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), - ImmutableList.of(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), - ImmutableList.of(p.symbol("RIGHT_GROUP_BY")), - Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))), - Optional.of(p.symbol("LEFT_HASH")), - Optional.of(p.symbol("RIGHT_HASH")))) - .addAggregation(p.symbol("AVG", DOUBLE), PlanBuilder.aggregation("AVG", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"))), ImmutableList.of(DOUBLE)) - .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("RIGHT_GROUP_BY")) + ImmutableList.of(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + ImmutableList.of(), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))))) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("AVG", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI")) .step(PARTIAL))) .matches(project(ImmutableMap.of( "LEFT_GROUP_BY", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_GROUP_BY")), - "RIGHT_GROUP_BY", PlanMatchPattern.expression(new Reference(BIGINT, "RIGHT_GROUP_BY")), + "LEFT_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_EQUI")), + "LEFT_NON_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_NON_EQUI")), "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), join(INNER, builder -> builder .equiCriteria("LEFT_EQUI", "RIGHT_EQUI") .filter(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))) .left( aggregation( - singleGroupingSet("LEFT_EQUI", "LEFT_NON_EQUI", "LEFT_GROUP_BY", "LEFT_HASH"), + singleGroupingSet("LEFT_GROUP_BY", "LEFT_EQUI", "LEFT_NON_EQUI"), ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("LEFT_AGGR"))), Optional.empty(), PARTIAL, - values("LEFT_EQUI", "LEFT_NON_EQUI", "LEFT_GROUP_BY", "LEFT_AGGR", "LEFT_HASH"))) + values("LEFT_EQUI", "LEFT_NON_EQUI", "LEFT_GROUP_BY", "LEFT_AGGR"))) + .right( + values("RIGHT_EQUI", "RIGHT_NON_EQUI"))))); + + // push to right child + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI"), p.symbol("RIGHT_GROUP_BY"), p.symbol("RIGHT_AGGR")), + ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), + ImmutableList.of(), + ImmutableList.of(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI"), p.symbol("RIGHT_GROUP_BY"), p.symbol("RIGHT_AGGR")), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))))) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "RIGHT_AGGR"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("RIGHT_GROUP_BY"), p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")) + .step(PARTIAL))) + .matches(project(ImmutableMap.of( + "RIGHT_GROUP_BY", PlanMatchPattern.expression(new Reference(BIGINT, "RIGHT_GROUP_BY")), + "RIGHT_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "RIGHT_EQUI")), + "RIGHT_NON_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "RIGHT_NON_EQUI")), + "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), + join(INNER, builder -> builder + .equiCriteria("LEFT_EQUI", "RIGHT_EQUI") + .filter(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))) + .left( + values("LEFT_EQUI", "LEFT_NON_EQUI")) + .right( + aggregation( + singleGroupingSet("RIGHT_GROUP_BY", "RIGHT_EQUI", "RIGHT_NON_EQUI"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("RIGHT_AGGR"))), + Optional.empty(), + PARTIAL, + values("RIGHT_EQUI", "RIGHT_NON_EQUI", "RIGHT_GROUP_BY", "RIGHT_AGGR")))))); + } + + @Test + public void testDoesNotPushPartialAggregationForExpandingJoin() + { + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .overrideStats(CHILD_ID.toString(), new PlanNodeStatsEstimate(10.0, ImmutableMap.of())) + .overrideStats(JOIN_ID.toString(), new PlanNodeStatsEstimate(20.0, ImmutableMap.of())) + .on(p -> p.aggregation(ab -> ab + .source( + p.join(JOIN_ID, + INNER, + p.values(CHILD_ID, p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")), + ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), + ImmutableList.of(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + ImmutableList.of(), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))), + Optional.empty(), + Optional.empty(), + Optional.empty(), + ImmutableMap.of())) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI")) + .step(PARTIAL))) + .doesNotFire(); + } + + @Test + public void testDoesNotPushPartialAggregationIfPushedGroupingSetIsLarger() + { + // partial aggregation should not be pushed down because it would require extra grouping symbols + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")), + ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), + ImmutableList.of(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + ImmutableList.of(), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))))) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI")) + .step(PARTIAL))) + .doesNotFire(); + + // partial aggregation should not be pushed down because it would require extra grouping symbols (with projection) + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.project( + Assignments.builder() + .put(p.symbol("LEFT_AGGR_PRJ"), new Call(ADD_BIGINT, ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"), new Reference(BIGINT, "LEFT_AGGR")))) + .putIdentity(p.symbol("LEFT_GROUP_BY")) + .putIdentity(p.symbol("LEFT_EQUI")) + .putIdentity(p.symbol("LEFT_NON_EQUI")) + .build(), + p.join( + INNER, + p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")), + ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), + ImmutableList.of(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + ImmutableList.of(), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI")))))) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR_PRJ"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI")) + .step(PARTIAL))) + .doesNotFire(); + } + + @Test + public void testDoesNotPushPartialAggregationIfPushedGroupingSetIsSame() + { + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + p.values(p.symbol("DATE_DIM_DATE_ID"), p.symbol("DATE_DIM_YEAR")), + ImmutableList.of(new EquiJoinClause(p.symbol("FACT_DATE_ID"), p.symbol("DATE_DIM_DATE_ID"))), + ImmutableList.of(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + ImmutableList.of(p.symbol("DATE_DIM_YEAR")), + Optional.empty())) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "AMOUNT"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("DATE_DIM_YEAR")) + .step(PARTIAL))) + .matches(project(ImmutableMap.of( + "DATE_DIM_YEAR", PlanMatchPattern.expression(new Reference(BIGINT, "DATE_DIM_YEAR")), + "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), + join(INNER, builder -> builder + .equiCriteria("FACT_DATE_ID", "DATE_DIM_DATE_ID") + .left( + aggregation( + singleGroupingSet("FACT_DATE_ID"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("AMOUNT"))), + Optional.empty(), + PARTIAL, + values("FACT_DATE_ID", "AMOUNT"))) + .right( + values("DATE_DIM_DATE_ID", "DATE_DIM_YEAR"))))); + } + + @Test + public void testDoesNotPushPartialAggregationIfGroupingSymbolHasBigNDV() + { + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .overrideStats( + CHILD_ID.toString(), + new PlanNodeStatsEstimate(10.0, ImmutableMap.of( + new Symbol(BIGINT, "FACT_DATE_ID"), new SymbolStatsEstimate(NaN, NaN, 0.0, NaN, 10.0)))) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(CHILD_ID, p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + p.values(p.symbol("DATE_DIM_DATE_ID"), p.symbol("DATE_DIM_YEAR")), + ImmutableList.of(new EquiJoinClause(p.symbol("FACT_DATE_ID"), p.symbol("DATE_DIM_DATE_ID"))), + ImmutableList.of(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + ImmutableList.of(p.symbol("DATE_DIM_YEAR")), + Optional.empty())) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "AMOUNT"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("DATE_DIM_YEAR")) + .step(PARTIAL))) + .doesNotFire(); + } + + @Test + public void testKeepsIntermediateAggregation() + { + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + p.values(p.symbol("DATE_DIM_DATE_ID"), p.symbol("DATE_DIM_YEAR")), + ImmutableList.of(new EquiJoinClause(p.symbol("FACT_DATE_ID"), p.symbol("DATE_DIM_DATE_ID"))), + ImmutableList.of(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + ImmutableList.of(p.symbol("DATE_DIM_YEAR")), + Optional.empty())) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "AMOUNT"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("DATE_DIM_YEAR")) + .step(PARTIAL) + .exchangeInputAggregation(true))) + .matches( + aggregation( + singleGroupingSet("DATE_DIM_YEAR"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("AVG"))), + Optional.empty(), + INTERMEDIATE, + project(ImmutableMap.of( + "DATE_DIM_YEAR", PlanMatchPattern.expression(new Reference(BIGINT, "DATE_DIM_YEAR")), + "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), + join(INNER, builder -> builder + .equiCriteria("FACT_DATE_ID", "DATE_DIM_DATE_ID") + .left( + aggregation( + singleGroupingSet("FACT_DATE_ID"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("AMOUNT"))), + Optional.empty(), + PARTIAL, + values("FACT_DATE_ID", "AMOUNT"))) + .right( + values("DATE_DIM_DATE_ID", "DATE_DIM_YEAR")))))); + + // intermediate aggregation should not be added if pushed aggregation has same (in terms of symbols) or smaller grouping set + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithoutProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.join( + INNER, + p.values(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + p.values(p.symbol("DATE_DIM_DATE_ID"), p.symbol("DATE_DIM_YEAR")), + ImmutableList.of(new EquiJoinClause(p.symbol("FACT_DATE_ID"), p.symbol("DATE_DIM_DATE_ID"))), + ImmutableList.of(p.symbol("FACT_DATE_ID"), p.symbol("AMOUNT")), + ImmutableList.of(p.symbol("DATE_DIM_YEAR")), + Optional.empty())) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "AMOUNT"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("FACT_DATE_ID")) + .step(PARTIAL) + .exchangeInputAggregation(true))) + .matches(project(ImmutableMap.of( + "FACT_DATE_ID", PlanMatchPattern.expression(new Reference(BIGINT, "FACT_DATE_ID")), + "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), + join(INNER, builder -> builder + .equiCriteria("FACT_DATE_ID", "DATE_DIM_DATE_ID") + .left( + aggregation( + singleGroupingSet("FACT_DATE_ID"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("AMOUNT"))), + Optional.empty(), + PARTIAL, + values("FACT_DATE_ID", "AMOUNT"))) + .right( + values("DATE_DIM_DATE_ID", "DATE_DIM_YEAR"))))); + } + + @Test + public void testPushesPartialAggregationThroughJoinWithProjection() + { + tester().assertThat(new PushPartialAggregationThroughJoin().pushPartialAggregationThroughJoinWithProjection()) + .on(p -> p.aggregation(ab -> ab + .source( + p.project( + Assignments.builder() + .put(p.symbol("LEFT_AGGR_PRJ"), new Call(ADD_BIGINT, ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"), new Reference(BIGINT, "LEFT_AGGR")))) + .putIdentity(p.symbol("LEFT_GROUP_BY")) + .putIdentity(p.symbol("LEFT_EQUI")) + .putIdentity(p.symbol("LEFT_NON_EQUI")) + .build(), + p.join( + INNER, + p.values(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + p.values(p.symbol("RIGHT_EQUI"), p.symbol("RIGHT_NON_EQUI")), + ImmutableList.of(new EquiJoinClause(p.symbol("LEFT_EQUI"), p.symbol("RIGHT_EQUI"))), + ImmutableList.of(p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI"), p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_AGGR")), + ImmutableList.of(), + Optional.of(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI")))))) + .addAggregation(p.symbol("AVG", DOUBLE), aggregation("avg", ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR_PRJ"))), ImmutableList.of(DOUBLE)) + .singleGroupingSet(p.symbol("LEFT_GROUP_BY"), p.symbol("LEFT_EQUI"), p.symbol("LEFT_NON_EQUI")) + .step(PARTIAL))) + .matches(project(ImmutableMap.of( + "LEFT_GROUP_BY", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_GROUP_BY")), + "LEFT_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_EQUI")), + "LEFT_NON_EQUI", PlanMatchPattern.expression(new Reference(BIGINT, "LEFT_NON_EQUI")), + "AVG", PlanMatchPattern.expression(new Reference(DOUBLE, "AVG"))), + join(INNER, builder -> builder + .equiCriteria("LEFT_EQUI", "RIGHT_EQUI") + .filter(new Comparison(LESS_THAN_OR_EQUAL, new Reference(BIGINT, "LEFT_NON_EQUI"), new Reference(BIGINT, "RIGHT_NON_EQUI"))) + .left( + aggregation( + singleGroupingSet("LEFT_GROUP_BY", "LEFT_EQUI", "LEFT_NON_EQUI"), + ImmutableMap.of(Optional.of("AVG"), aggregationFunction("avg", ImmutableList.of("LEFT_AGGR_PRJ"))), + Optional.empty(), + PARTIAL, + project( + ImmutableMap.of("LEFT_AGGR_PRJ", PlanMatchPattern.expression(new Call(ADD_BIGINT, ImmutableList.of(new Reference(BIGINT, "LEFT_AGGR"), new Reference(BIGINT, "LEFT_AGGR"))))), + values("LEFT_EQUI", "LEFT_NON_EQUI", "LEFT_GROUP_BY", "LEFT_AGGR")))) .right( - values("RIGHT_EQUI", "RIGHT_NON_EQUI", "RIGHT_GROUP_BY", "RIGHT_HASH"))))); + project( + values("RIGHT_EQUI", "RIGHT_NON_EQUI")))))); } } diff --git a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java index 73d17730ff34..7563d5259853 100644 --- a/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java +++ b/core/trino-main/src/test/java/io/trino/sql/planner/iterative/rule/test/PlanBuilder.java @@ -422,6 +422,7 @@ public class AggregationBuilder private Optional hashSymbol = Optional.empty(); private Optional groupIdSymbol = Optional.empty(); private Optional nodeId = Optional.empty(); + private Optional exchangeInputAggregation = Optional.empty(); public AggregationBuilder source(PlanNode source) { @@ -501,6 +502,12 @@ public AggregationBuilder nodeId(PlanNodeId nodeId) return this; } + public AggregationBuilder exchangeInputAggregation(boolean exchangeInputAggregation) + { + this.exchangeInputAggregation = Optional.of(exchangeInputAggregation); + return this; + } + protected AggregationNode build() { checkState(groupingSets != null, "No grouping sets defined; use globalGrouping/groupingKeys method"); @@ -512,7 +519,8 @@ protected AggregationNode build() preGroupedSymbols, step, hashSymbol, - groupIdSymbol); + groupIdSymbol, + exchangeInputAggregation); } } @@ -1085,9 +1093,37 @@ public JoinNode join( Optional rightHashSymbol, Optional distributionType, Map dynamicFilters) + { + return join(idAllocator.getNextId(), + type, + left, + right, + criteria, + leftOutputSymbols, + rightOutputSymbols, + filter, + leftHashSymbol, + rightHashSymbol, + distributionType, + dynamicFilters); + } + + public JoinNode join( + PlanNodeId id, + JoinType type, + PlanNode left, + PlanNode right, + List criteria, + List leftOutputSymbols, + List rightOutputSymbols, + Optional filter, + Optional leftHashSymbol, + Optional rightHashSymbol, + Optional distributionType, + Map dynamicFilters) { return new JoinNode( - idAllocator.getNextId(), + id, type, left, right, 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 cde6875d5dd0..939852a3ce79 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 @@ -7,13 +7,15 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_day_name, d_week_seq) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_week_seq]) - partial aggregation over (d_day_name, d_week_seq) + intermediate aggregation over (d_day_name, d_week_seq) join (INNER, REPLICATED): local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ws_sold_date_sk::EQUAL) - scan web_sales - dynamic filter (cs_sold_date_sk::EQUAL) - scan catalog_sales + partial aggregation over (ws_sold_date_sk) + dynamic filter (ws_sold_date_sk::EQUAL) + scan web_sales + partial aggregation over (cs_sold_date_sk) + dynamic filter (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) @@ -29,13 +31,15 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_day_name_134, d_week_seq_124) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_week_seq_124]) - partial aggregation over (d_day_name_134, d_week_seq_124) + intermediate 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) - scan web_sales - dynamic filter (cs_sold_date_sk_117::EQUAL) - scan catalog_sales + partial aggregation over (ws_sold_date_sk_81) + dynamic filter (ws_sold_date_sk_81::EQUAL) + scan web_sales + partial aggregation over (cs_sold_date_sk_117) + dynamic filter (cs_sold_date_sk_117::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) dynamic filter (d_week_seq_124::EQUAL) 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 ec8d79e0127f..d05c944ec5c4 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 @@ -5,11 +5,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_year, i_brand, i_brand_id]) - partial aggregation over (d_year, i_brand, i_brand_id) + intermediate 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) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 48c371873112..c18eec14677f 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 @@ -8,15 +8,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_513, c_customer_id_500, c_email_address_515, c_first_name_507, c_last_name_508, c_login_514, c_preferred_cust_flag_509, d_year_561) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_500]) - partial aggregation over (c_birth_country_513, c_customer_id_500, c_email_address_515, c_first_name_507, c_last_name_508, c_login_514, c_preferred_cust_flag_509, d_year_561) + intermediate aggregation over (c_birth_country_513, c_customer_id_500, c_email_address_515, c_first_name_507, c_last_name_508, c_login_514, c_preferred_cust_flag_509, d_year_561) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_521]) - join (INNER, REPLICATED): - dynamic filter (cs_bill_customer_sk_521::EQUAL, cs_sold_date_sk_552::EQUAL) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk_521, d_year_561) + remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_521]) + join (INNER, REPLICATED): + dynamic filter (cs_bill_customer_sk_521::EQUAL, cs_sold_date_sk_552::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim 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) @@ -24,15 +25,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_1154, c_customer_id_1141, c_email_address_1156, c_first_name_1148, c_last_name_1149, c_login_1155, c_preferred_cust_flag_1150, d_year_1202) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_1141]) - partial aggregation over (c_birth_country_1154, c_customer_id_1141, c_email_address_1156, c_first_name_1148, c_last_name_1149, c_login_1155, c_preferred_cust_flag_1150, d_year_1202) + intermediate aggregation over (c_birth_country_1154, c_customer_id_1141, c_email_address_1156, c_first_name_1148, c_last_name_1149, c_login_1155, c_preferred_cust_flag_1150, d_year_1202) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1163]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_1163::EQUAL, ws_sold_date_sk_1193::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_1202, ws_bill_customer_sk_1163) + remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1163]) + join (INNER, REPLICATED): + dynamic filter (ws_bill_customer_sk_1163::EQUAL, ws_sold_date_sk_1193::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_1140]) dynamic filter (c_customer_id_1141::EQUAL, c_customer_id_1141::EQUAL) @@ -40,15 +42,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_159, c_customer_id_146, c_email_address_161, c_first_name_153, c_last_name_154, c_login_160, c_preferred_cust_flag_155, d_year_196) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_146]) - partial aggregation over (c_birth_country_159, c_customer_id_146, c_email_address_161, c_first_name_153, c_last_name_154, c_login_160, c_preferred_cust_flag_155, d_year_196) + intermediate aggregation over (c_birth_country_159, c_customer_id_146, c_email_address_161, c_first_name_153, c_last_name_154, c_login_160, c_preferred_cust_flag_155, d_year_196) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk_167]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk_167::EQUAL, ss_sold_date_sk_187::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_196, ss_customer_sk_167) + remote exchange (REPARTITION, HASH, [ss_customer_sk_167]) + join (INNER, REPLICATED): + dynamic filter (ss_customer_sk_167::EQUAL, ss_sold_date_sk_187::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_145]) dynamic filter (c_customer_id_146::EQUAL) @@ -58,15 +61,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_789, c_customer_id_776, c_email_address_791, c_first_name_783, c_last_name_784, c_login_790, c_preferred_cust_flag_785, d_year_837) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_776]) - partial aggregation over (c_birth_country_789, c_customer_id_776, c_email_address_791, c_first_name_783, c_last_name_784, c_login_790, c_preferred_cust_flag_785, d_year_837) + intermediate aggregation over (c_birth_country_789, c_customer_id_776, c_email_address_791, c_first_name_783, c_last_name_784, c_login_790, c_preferred_cust_flag_785, d_year_837) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_797]) - join (INNER, REPLICATED): - dynamic filter (cs_bill_customer_sk_797::EQUAL, cs_sold_date_sk_828::EQUAL) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_bill_customer_sk_797, d_year_837) + remote exchange (REPARTITION, HASH, [cs_bill_customer_sk_797]) + join (INNER, REPLICATED): + dynamic filter (cs_bill_customer_sk_797::EQUAL, cs_sold_date_sk_828::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_775]) dynamic filter (c_customer_id_776::EQUAL, c_customer_id_776::EQUAL) @@ -74,15 +78,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_1430, c_customer_id_1417, c_email_address_1432, c_first_name_1424, c_last_name_1425, c_login_1431, c_preferred_cust_flag_1426, d_year_1478) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_1417]) - partial aggregation over (c_birth_country_1430, c_customer_id_1417, c_email_address_1432, c_first_name_1424, c_last_name_1425, c_login_1431, c_preferred_cust_flag_1426, d_year_1478) + intermediate aggregation over (c_birth_country_1430, c_customer_id_1417, c_email_address_1432, c_first_name_1424, c_last_name_1425, c_login_1431, c_preferred_cust_flag_1426, d_year_1478) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1439]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_1439::EQUAL, ws_sold_date_sk_1469::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_1478, ws_bill_customer_sk_1439) + remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_1439]) + join (INNER, REPLICATED): + dynamic filter (ws_bill_customer_sk_1439::EQUAL, ws_sold_date_sk_1469::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_1416]) dynamic filter (c_customer_id_1417::EQUAL) @@ -90,15 +95,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + intermediate aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year, ss_customer_sk) + remote exchange (REPARTITION, HASH, [ss_customer_sk]) + join (INNER, REPLICATED): + dynamic filter (ss_customer_sk::EQUAL, 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, [c_customer_sk]) scan customer 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..0983e18133ae 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 @@ -10,57 +10,60 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_store_id]) - partial aggregation over (s_store_id) + intermediate aggregation over (s_store_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): - local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) - scan store_sales - dynamic filter (sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (store_sk) + join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + scan store_sales + dynamic filter (sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store final aggregation over (cp_catalog_page_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cp_catalog_page_id]) - partial aggregation over (cp_catalog_page_id) + intermediate aggregation over (cp_catalog_page_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): - local exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (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) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (page_sk) + join (INNER, REPLICATED): + local exchange (REPARTITION, ROUND_ROBIN, []) + dynamic filter (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) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan catalog_page final aggregation over (web_site_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [web_site_id]) - partial aggregation over (web_site_id) + intermediate aggregation over (web_site_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): - remote exchange (REPARTITION, ROUND_ROBIN, []) - dynamic filter (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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) - dynamic filter (wr_returned_date_sk::EQUAL) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (wsr_web_site_sk) + join (INNER, REPLICATED): + remote exchange (REPARTITION, ROUND_ROBIN, []) + dynamic filter (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) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) + dynamic filter (wr_returned_date_sk::EQUAL) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan web_site 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 5f38b199b86f..77f98beb2516 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 @@ -5,22 +5,23 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id]) - partial aggregation over (i_item_id) + intermediate aggregation over (i_item_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk) 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) - scan store_sales + join (INNER, REPLICATED): + dynamic filter (ss_cdemo_sk::EQUAL, ss_item_sk::EQUAL, ss_promo_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 1cde47b7a3d3..cdcc5eb6a40d 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 @@ -7,15 +7,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_84, c_customer_id_71, c_email_address_86, c_first_name_78, c_last_name_79, c_login_85, c_preferred_cust_flag_80, d_year_121) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_71]) - partial aggregation over (c_birth_country_84, c_customer_id_71, c_email_address_86, c_first_name_78, c_last_name_79, c_login_85, c_preferred_cust_flag_80, d_year_121) + intermediate aggregation over (c_birth_country_84, c_customer_id_71, c_email_address_86, c_first_name_78, c_last_name_79, c_login_85, c_preferred_cust_flag_80, d_year_121) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk_92]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk_92::EQUAL, ss_sold_date_sk_112::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_121, ss_customer_sk_92) + remote exchange (REPARTITION, HASH, [ss_customer_sk_92]) + join (INNER, REPLICATED): + dynamic filter (ss_customer_sk_92::EQUAL, ss_sold_date_sk_112::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_70]) dynamic filter (c_customer_id_71::EQUAL, c_customer_id_71::EQUAL) @@ -23,15 +24,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_339, c_customer_id_326, c_email_address_341, c_first_name_333, c_last_name_334, c_login_340, c_preferred_cust_flag_335, d_year_387) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_326]) - partial aggregation over (c_birth_country_339, c_customer_id_326, c_email_address_341, c_first_name_333, c_last_name_334, c_login_340, c_preferred_cust_flag_335, d_year_387) + intermediate aggregation over (c_birth_country_339, c_customer_id_326, c_email_address_341, c_first_name_333, c_last_name_334, c_login_340, c_preferred_cust_flag_335, d_year_387) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_348]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_348::EQUAL, ws_sold_date_sk_378::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_387, ws_bill_customer_sk_348) + remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_348]) + join (INNER, REPLICATED): + dynamic filter (ws_bill_customer_sk_348::EQUAL, ws_sold_date_sk_378::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_325]) dynamic filter (c_customer_id_326::EQUAL) @@ -40,15 +42,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id]) - partial aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) + intermediate aggregation over (c_birth_country, c_customer_id, c_email_address, c_first_name, c_last_name, c_login, c_preferred_cust_flag, d_year) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year, ss_customer_sk) + remote exchange (REPARTITION, HASH, [ss_customer_sk]) + join (INNER, REPLICATED): + dynamic filter (ss_customer_sk::EQUAL, 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, [c_customer_sk]) dynamic filter (c_customer_id::EQUAL) @@ -56,15 +59,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_birth_country_516, c_customer_id_503, c_email_address_518, c_first_name_510, c_last_name_511, c_login_517, c_preferred_cust_flag_512, d_year_564) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_503]) - partial aggregation over (c_birth_country_516, c_customer_id_503, c_email_address_518, c_first_name_510, c_last_name_511, c_login_517, c_preferred_cust_flag_512, d_year_564) + intermediate aggregation over (c_birth_country_516, c_customer_id_503, c_email_address_518, c_first_name_510, c_last_name_511, c_login_517, c_preferred_cust_flag_512, d_year_564) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_525]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_525::EQUAL, ws_sold_date_sk_555::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (d_year_564, ws_bill_customer_sk_525) + remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_525]) + join (INNER, REPLICATED): + dynamic filter (ws_bill_customer_sk_525::EQUAL, ws_sold_date_sk_555::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_502]) scan customer 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 fb4a7ddc7d1f..d24d160fee56 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 @@ -7,14 +7,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_category, i_class, i_current_price, i_item_desc, i_item_id]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + intermediate 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) scan item 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 823730f6cf64..a99d8a7e64c1 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 @@ -11,14 +11,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_6, i_category_id_8, i_class_id_7]) - partial aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) + intermediate aggregation over (i_brand_id_6, i_category_id_8, i_class_id_7) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): @@ -36,38 +37,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (brand_id, category_id, class_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_72, i_category_id_76, i_class_id_74]) - partial aggregation over (i_brand_id_72, i_category_id_76, i_class_id_74) + intermediate 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_41) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk_41::EQUAL, ss_sold_date_sk_62::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_128, i_category_id_132, i_class_id_130]) - partial aggregation over (i_brand_id_128, i_category_id_132, i_class_id_130) + intermediate 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_item_sk) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_184, i_category_id_188, i_class_id_186]) - partial aggregation over (i_brand_id_184, i_category_id_188, i_class_id_186) + intermediate 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) scan item @@ -101,14 +105,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_469, i_category_id_473, i_class_id_471]) - partial aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) + intermediate aggregation over (i_brand_id_469, i_category_id_473, i_class_id_471) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (cs_item_sk_440::EQUAL, cs_sold_date_sk_459::EQUAL) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_item_sk_440) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk_440::EQUAL, cs_sold_date_sk_459::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): @@ -126,38 +131,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (brand_id_542, category_id_544, class_id_543) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_580, i_category_id_584, i_class_id_582]) - partial aggregation over (i_brand_id_580, i_category_id_584, i_class_id_582) + intermediate 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_549) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk_549::EQUAL, ss_sold_date_sk_570::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_670, i_category_id_674, i_class_id_672]) - partial aggregation over (i_brand_id_670, i_category_id_674, i_class_id_672) + intermediate 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_item_sk_641) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk_641::EQUAL, cs_sold_date_sk_660::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_760, i_category_id_764, i_class_id_762]) - partial aggregation over (i_brand_id_760, i_category_id_764, i_class_id_762) + intermediate 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk_719) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk_719::EQUAL, ws_sold_date_sk_750::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -191,14 +199,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand_id_1049, i_category_id_1053, i_class_id_1051) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_1049, i_category_id_1053, i_class_id_1051]) - partial aggregation over (i_brand_id_1049, i_category_id_1053, i_class_id_1051) + intermediate aggregation over (i_brand_id_1049, i_category_id_1053, i_class_id_1051) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (ws_item_sk_1008::EQUAL, ws_sold_date_sk_1039::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk_1008) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk_1008::EQUAL, ws_sold_date_sk_1039::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, PARTITIONED): @@ -216,38 +225,41 @@ local exchange (GATHER, SINGLE, []) final aggregation over (brand_id_1122, category_id_1124, class_id_1123) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand_id_1160, i_category_id_1164, i_class_id_1162]) - partial aggregation over (i_brand_id_1160, i_category_id_1164, i_class_id_1162) + intermediate 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk_1129) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk_1129::EQUAL, ss_sold_date_sk_1150::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_1250, i_category_id_1254, i_class_id_1252]) - partial aggregation over (i_brand_id_1250, i_category_id_1254, i_class_id_1252) + intermediate 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_item_sk_1221) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk_1221::EQUAL, cs_sold_date_sk_1240::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item remote exchange (REPARTITION, HASH, [i_brand_id_1340, i_category_id_1344, i_class_id_1342]) - partial aggregation over (i_brand_id_1340, i_category_id_1344, i_class_id_1342) + intermediate 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk_1299) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk_1299::EQUAL, ws_sold_date_sk_1330::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 1e38c40aae0e..89e0e153ba2e 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 @@ -7,14 +7,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_category, i_class, i_current_price, i_item_desc, i_item_id]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + intermediate 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_item_sk) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 85d20da0e082..43bfe1d71acb 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 @@ -5,15 +5,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id, w_warehouse_name]) - partial aggregation over (i_item_id, w_warehouse_name) + intermediate aggregation over (i_item_id, w_warehouse_name) join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL) - scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (inv_item_sk, inv_warehouse_sk) + join (INNER, REPLICATED): + dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL) + scan inventory + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 ec138e78f767..a1ef4bdfe553 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 @@ -9,18 +9,18 @@ final aggregation over () final aggregation over (d_date_6, gid, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk]) - partial aggregation over (d_date_6, gid, ss_item_sk) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk, ss_sold_date_sk) dynamic filter (ss_item_sk::EQUAL, ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + dynamic filter (i_item_sk::EQUAL) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_item_sk]) join (INNER, REPLICATED): @@ -33,14 +33,14 @@ final aggregation over () cross join (can skip output duplicates): final aggregation over (ss_customer_sk_43) local exchange (GATHER, SINGLE, []) - partial aggregation over (ss_customer_sk_43) - join (INNER, PARTITIONED): + join (INNER, PARTITIONED): + partial aggregation over (ss_customer_sk_43) remote exchange (REPARTITION, HASH, [ss_customer_sk_43]) dynamic filter (ss_customer_sk_43::EQUAL) scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [c_customer_sk]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -49,8 +49,8 @@ final aggregation over () partial aggregation over () final aggregation over (ss_customer_sk_71) local exchange (GATHER, SINGLE, []) - partial aggregation over (ss_customer_sk_71) - join (INNER, PARTITIONED): + join (INNER, PARTITIONED): + partial aggregation over (ss_customer_sk_71) remote exchange (REPARTITION, HASH, [ss_customer_sk_71]) join (INNER, REPLICATED): dynamic filter (ss_customer_sk_71::EQUAL, ss_sold_date_sk_91::EQUAL) @@ -58,9 +58,9 @@ final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk_94]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [c_customer_sk_94]) + scan customer partial aggregation over () join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk]) @@ -69,18 +69,18 @@ final aggregation over () final aggregation over (d_date_210, gid_263, ss_item_sk_184) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk_184]) - partial aggregation over (d_date_210, gid_263, ss_item_sk_184) + join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk_184, ss_sold_date_sk_205) dynamic filter (ss_item_sk_184::EQUAL, ss_item_sk_184::EQUAL, ss_sold_date_sk_205::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 (i_item_sk_238::EQUAL) - scan item + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + dynamic filter (i_item_sk_238::EQUAL) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_item_sk]) join (INNER, REPLICATED): @@ -93,14 +93,14 @@ final aggregation over () cross join (can skip output duplicates): final aggregation over (ss_customer_sk_271) local exchange (GATHER, SINGLE, []) - partial aggregation over (ss_customer_sk_271) - join (INNER, PARTITIONED): + join (INNER, PARTITIONED): + partial aggregation over (ss_customer_sk_271) remote exchange (REPARTITION, HASH, [ss_customer_sk_271]) dynamic filter (ss_customer_sk_271::EQUAL) scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk_294]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [c_customer_sk_294]) + scan customer local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) final aggregation over () @@ -109,8 +109,8 @@ final aggregation over () partial aggregation over () final aggregation over (ss_customer_sk_318) local exchange (GATHER, SINGLE, []) - partial aggregation over (ss_customer_sk_318) - join (INNER, PARTITIONED): + join (INNER, PARTITIONED): + partial aggregation over (ss_customer_sk_318) remote exchange (REPARTITION, HASH, [ss_customer_sk_318]) join (INNER, REPLICATED): dynamic filter (ss_customer_sk_318::EQUAL, ss_sold_date_sk_338::EQUAL) @@ -118,6 +118,6 @@ final aggregation over () local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk_341]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [c_customer_sk_341]) + scan customer 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 a01a09cafc2e..e6c9081bd421 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 @@ -5,22 +5,23 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id]) - partial aggregation over (i_item_id) + intermediate aggregation over (i_item_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk) 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) - scan catalog_sales + join (INNER, REPLICATED): + dynamic filter (cs_bill_cdemo_sk::EQUAL, cs_item_sk::EQUAL, cs_promo_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan customer_demographics + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan promotion + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 a846fed83ea1..ba2cec656b54 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 @@ -7,15 +7,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county_137, d_qoy_110, d_year_106) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county_137]) - partial aggregation over (ca_county_137, d_qoy_110, d_year_106) + intermediate aggregation over (ca_county_137, d_qoy_110, d_year_106) 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy_110, d_year_106, ss_addr_sk_80) + join (INNER, REPLICATED): + partial aggregation over (ss_addr_sk_80, ss_sold_date_sk_97) + dynamic filter (ss_addr_sk_80::EQUAL, 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) @@ -23,15 +25,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county_185, d_qoy_158, d_year_154) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county_185]) - partial aggregation over (ca_county_185, d_qoy_158, d_year_154) + intermediate aggregation over (ca_county_185, d_qoy_158, d_year_154) 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy_158, d_year_154, ws_bill_addr_sk) + join (INNER, REPLICATED): + partial aggregation over (ws_bill_addr_sk, ws_sold_date_sk) + dynamic filter (ws_bill_addr_sk::EQUAL, 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) @@ -40,15 +44,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county_267, d_qoy_240, d_year_236) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county_267]) - partial aggregation over (ca_county_267, d_qoy_240, d_year_236) + intermediate aggregation over (ca_county_267, d_qoy_240, d_year_236) 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy_240, d_year_236, ws_bill_addr_sk_200) + join (INNER, REPLICATED): + partial aggregation over (ws_bill_addr_sk_200, ws_sold_date_sk_227) + dynamic filter (ws_bill_addr_sk_200::EQUAL, 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, ca_county_267::EQUAL) @@ -56,15 +62,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county_349, d_qoy_322, d_year_318) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county_349]) - partial aggregation over (ca_county_349, d_qoy_322, d_year_318) + intermediate aggregation over (ca_county_349, d_qoy_322, d_year_318) 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy_322, d_year_318, ws_bill_addr_sk_282) + join (INNER, REPLICATED): + partial aggregation over (ws_bill_addr_sk_282, ws_sold_date_sk_309) + dynamic filter (ws_bill_addr_sk_282::EQUAL, ws_sold_date_sk_309::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_342]) dynamic filter (ca_county_349::EQUAL) @@ -73,15 +81,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county, d_qoy, d_year) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county]) - partial aggregation over (ca_county, d_qoy, d_year) + intermediate aggregation over (ca_county, d_qoy, d_year) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_addr_sk]) - join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy, d_year, ss_addr_sk) + join (INNER, REPLICATED): + partial aggregation over (ss_addr_sk, ss_sold_date_sk) + dynamic filter (ss_addr_sk::EQUAL, 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) @@ -89,15 +99,17 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_county_66, d_qoy_39, d_year_35) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_county_66]) - partial aggregation over (ca_county_66, d_qoy_39, d_year_35) + intermediate aggregation over (ca_county_66, d_qoy_39, d_year_35) 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_qoy_39, d_year_35, ss_addr_sk_9) + join (INNER, REPLICATED): + partial aggregation over (ss_addr_sk_9, ss_sold_date_sk_26) + dynamic filter (ss_addr_sk_9::EQUAL, 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]) scan customer_address 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 bf7539946877..72c962ce704c 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 @@ -8,18 +8,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_manufact_id_6]) - partial aggregation over (i_manufact_id_6) + intermediate aggregation over (i_manufact_id_6) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales + join (INNER, REPLICATED): + dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -36,18 +37,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_95) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_manufact_id_95]) - partial aggregation over (i_manufact_id_95) + intermediate aggregation over (i_manufact_id_95) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) - scan catalog_sales + join (INNER, REPLICATED): + dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -64,18 +66,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_manufact_id_195) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_manufact_id_195]) - partial aggregation over (i_manufact_id_195) + intermediate aggregation over (i_manufact_id_195) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) - scan web_sales + join (INNER, REPLICATED): + dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): 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..66612900fd6c 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 @@ -8,15 +8,17 @@ final aggregation over () final aggregation over (c_first_name_6, c_last_name_7, d_date_3) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_6, c_last_name_7, d_date_3]) - partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) + intermediate aggregation over (c_first_name_6, c_last_name_7, d_date_3) 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_date_3, ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (ss_customer_sk, ss_sold_date_sk) + dynamic filter (ss_customer_sk::EQUAL, 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, [c_customer_sk]) scan customer @@ -24,15 +26,17 @@ final aggregation over () final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_50, c_last_name_51, d_date_14]) - partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) + intermediate aggregation over (c_first_name_50, c_last_name_51, d_date_14) 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (cs_bill_customer_sk, d_date_14) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (cs_bill_customer_sk, cs_sold_date_sk) + dynamic filter (cs_bill_customer_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_42]) scan customer @@ -40,15 +44,17 @@ final aggregation over () final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_102, c_last_name_103, d_date_66]) - partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) + intermediate aggregation over (c_first_name_102, c_last_name_103, d_date_66) 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_date_66, ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (ws_bill_customer_sk, ws_sold_date_sk) + dynamic filter (ws_bill_customer_sk::EQUAL, 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, [c_customer_sk_94]) scan customer 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 6be214cdab28..fbbab43a4284 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 @@ -6,40 +6,40 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name]) - partial aggregation over (d_moy, inv_item_sk, inv_warehouse_sk, w_warehouse_name) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (inv_date_sk, inv_item_sk, inv_warehouse_sk) dynamic filter (inv_date_sk::EQUAL, inv_item_sk::EQUAL, inv_item_sk::EQUAL, inv_warehouse_sk::EQUAL, inv_warehouse_sk::EQUAL) scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (w_warehouse_sk::EQUAL) - scan warehouse + dynamic filter (i_item_sk::EQUAL) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + dynamic filter (w_warehouse_sk::EQUAL) + scan warehouse local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [inv_item_sk_8, inv_warehouse_sk_9]) final aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40]) - partial aggregation over (d_moy_62, inv_item_sk_8, inv_warehouse_sk_9, w_warehouse_name_40) + join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (inv_date_sk_11, inv_item_sk_8, inv_warehouse_sk_9) dynamic filter (inv_date_sk_11::EQUAL, inv_item_sk_8::EQUAL, inv_warehouse_sk_9::EQUAL) scan inventory - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan item + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan warehouse + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan warehouse 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 6328a4ec63c0..67ea5d886aff 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 @@ -5,21 +5,22 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id, w_state) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id, w_state]) - partial aggregation over (i_item_id, w_state) + intermediate aggregation over (i_item_id, w_state) join (INNER, REPLICATED): join (INNER, REPLICATED): - 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) - scan catalog_sales + partial aggregation over (cs_item_sk, cs_warehouse_sk) + 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) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) + dynamic filter (cr_item_sk::EQUAL) + scan catalog_returns local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - dynamic filter (cr_item_sk::EQUAL) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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..c0894955c2b7 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 @@ -5,11 +5,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_category, i_category_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_year, i_category, i_category_id]) - partial aggregation over (d_year, i_category, i_category_id) + intermediate 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) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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..f63a39f5d838 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 @@ -9,11 +9,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_day_name, s_store_id, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_day_name, s_store_id, s_store_name]) - partial aggregation over (d_day_name, s_store_id, s_store_name) + intermediate 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) - scan store_sales + partial aggregation over (ss_sold_date_sk, ss_store_sk) + dynamic filter (ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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 f7a67cbd429e..79ec179ec3e7 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 @@ -12,9 +12,9 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) final aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_addr_sk]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_addr_sk]) + partial aggregation over (ss_addr_sk, ss_customer_sk, ss_ticket_number) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): @@ -29,9 +29,9 @@ local exchange (GATHER, SINGLE, []) local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [ca_address_sk]) - scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [ca_address_sk]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_11]) scan customer_address 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 46ef5f138643..12e565dad8ef 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 @@ -9,12 +9,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy, d_year, i_brand, i_category, s_company_name, s_store_name]) - partial aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) + intermediate aggregation over (d_moy, d_year, i_brand, i_category, s_company_name, s_store_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk, ss_store_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -31,12 +32,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90]) - partial aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) + intermediate aggregation over (d_moy_63, d_year_61, i_brand_14, i_category_18, s_company_name_102, s_store_name_90) 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) - scan store_sales + partial aggregation over (ss_item_sk_31, ss_sold_date_sk_52, ss_store_sk_36) + dynamic filter (ss_item_sk_31::EQUAL, ss_sold_date_sk_52::EQUAL, ss_store_sk_36::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -53,12 +55,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy_176, d_year_174, i_brand_127, i_category_131, s_company_name_215, s_store_name_203) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy_176, d_year_174, i_brand_127, i_category_131, s_company_name_215, s_store_name_203]) - partial aggregation over (d_moy_176, d_year_174, i_brand_127, i_category_131, s_company_name_215, s_store_name_203) + intermediate aggregation over (d_moy_176, d_year_174, i_brand_127, i_category_131, s_company_name_215, s_store_name_203) 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) - scan store_sales + partial aggregation over (ss_item_sk_144, ss_sold_date_sk_165, ss_store_sk_149) + dynamic filter (ss_item_sk_144::EQUAL, ss_sold_date_sk_165::EQUAL, ss_store_sk_149::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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 7b2b4b5f5285..dfbbd6c4e1ab 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 @@ -5,24 +5,25 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip]) - partial aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) + intermediate aggregation over (s_city, s_company_id, s_county, s_state, s_store_name, s_street_name, s_street_number, s_street_type, s_suite_number, s_zip) join (INNER, REPLICATED): - 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) - scan store_sales + partial aggregation over (ss_store_sk) + 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) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) + join (INNER, REPLICATED): + dynamic filter (sr_returned_date_sk::EQUAL) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [sr_customer_sk, sr_item_sk, sr_ticket_number]) - join (INNER, REPLICATED): - dynamic filter (sr_returned_date_sk::EQUAL) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store 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 c8929f941366..62b787dcd6af 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 @@ -8,20 +8,22 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_date_7, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_item_sk]) - partial aggregation over (d_date_7, ss_item_sk) + intermediate aggregation over (d_date_7, ss_item_sk) join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk::EQUAL) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk) + dynamic filter (ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim final aggregation over (d_date, ws_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_item_sk]) - partial aggregation over (d_date, ws_item_sk) + intermediate aggregation over (d_date, ws_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_sold_date_sk::EQUAL) - scan web_sales + partial aggregation over (ws_item_sk, ws_sold_date_sk) + dynamic filter (ws_sold_date_sk::EQUAL) + scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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..411deb735d9c 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 @@ -5,11 +5,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_year, i_brand, i_brand_id]) - partial aggregation over (d_year, i_brand, i_brand_id) + intermediate 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) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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..42d6f8b1dfab 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 @@ -5,14 +5,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand, i_brand_id]) - partial aggregation over (i_brand, i_brand_id) + intermediate aggregation over (i_brand, i_brand_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) scan item 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 3c8661a10af7..f12e28451de1 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 @@ -36,18 +36,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_83]) - partial aggregation over (i_item_id_83) + intermediate aggregation over (i_item_id_83) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) - scan catalog_sales + join (INNER, REPLICATED): + dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -64,18 +65,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_183) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_183]) - partial aggregation over (i_item_id_183) + intermediate aggregation over (i_item_id_183) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) - scan web_sales + join (INNER, REPLICATED): + dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): 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 2dc82a1cf9ee..daa1c57304ed 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 @@ -9,12 +9,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, d_moy, d_year, i_brand, i_category) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name, d_moy, d_year, i_brand, i_category]) - partial aggregation over (cc_name, d_moy, d_year, i_brand, i_category) + intermediate aggregation over (cc_name, d_moy, d_year, i_brand, i_category) 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) - scan catalog_sales + partial aggregation over (cs_call_center_sk, cs_item_sk, cs_sold_date_sk) + dynamic filter (cs_call_center_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -31,12 +32,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18]) - partial aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) + intermediate aggregation over (cc_name_102, d_moy_74, d_year_72, i_brand_14, i_category_18) 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) - scan catalog_sales + partial aggregation over (cs_call_center_sk_40, cs_item_sk_44, cs_sold_date_sk_63) + dynamic filter (cs_call_center_sk_40::EQUAL, cs_item_sk_44::EQUAL, cs_sold_date_sk_63::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -53,12 +55,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name_227, d_moy_199, d_year_197, i_brand_139, i_category_143) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name_227, d_moy_199, d_year_197, i_brand_139, i_category_143]) - partial aggregation over (cc_name_227, d_moy_199, d_year_197, i_brand_139, i_category_143) + intermediate aggregation over (cc_name_227, d_moy_199, d_year_197, i_brand_139, i_category_143) 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) - scan catalog_sales + partial aggregation over (cs_call_center_sk_165, cs_item_sk_169, cs_sold_date_sk_188) + dynamic filter (cs_call_center_sk_165::EQUAL, cs_item_sk_169::EQUAL, cs_sold_date_sk_188::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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 a5cb89ea2e78..ea83ae7c79ab 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 @@ -6,94 +6,97 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id]) - partial aggregation over (i_item_id) + intermediate aggregation over (i_item_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (d_date::EQUAL) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - 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) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - local exchange (GATHER, SINGLE, []) - remote exchange (GATHER, SINGLE, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id::EQUAL) - scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_69) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [i_item_id_69]) - partial aggregation over (i_item_id_69) - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk) join (INNER, REPLICATED): - dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) - scan catalog_sales + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - dynamic filter (d_date_94::EQUAL) + dynamic filter (d_date::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_126) + final aggregation over (d_date_6) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_date_126]) - partial aggregation over (d_date_126) + 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_128::EQUAL) + dynamic filter (d_week_seq_8::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_id_69::EQUAL) - scan item - final aggregation over (i_item_id_190) + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + dynamic filter (i_item_id::EQUAL) + scan item + join (INNER, PARTITIONED): + final aggregation over (i_item_id_69) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [i_item_id_190]) - partial aggregation over (i_item_id_190) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_item_sk]) + remote exchange (REPARTITION, HASH, [i_item_id_69]) + intermediate aggregation over (i_item_id_69) + join (INNER, REPLICATED): + partial aggregation over (cs_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) - scan web_sales + dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - dynamic filter (d_date_215::EQUAL) + dynamic filter (d_date_94::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_247) + final aggregation over (d_date_126) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_date_247]) - partial aggregation over (d_date_247) + 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_249::EQUAL) + dynamic filter (d_week_seq_128::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) local exchange (GATHER, SINGLE, []) remote exchange (GATHER, SINGLE, []) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + dynamic filter (i_item_id_69::EQUAL) + scan item + final aggregation over (i_item_id_190) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [i_item_id_190]) + intermediate aggregation over (i_item_id_190) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ws_item_sk]) + partial aggregation over (ws_item_sk) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + dynamic filter (d_date_215::EQUAL) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + 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) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + local exchange (GATHER, SINGLE, []) + remote exchange (GATHER, SINGLE, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_sk_189]) scan item 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 6f49127e1712..0c8ffaa835b4 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 @@ -13,10 +13,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_day_name, d_week_seq, ss_store_sk) 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) + intermediate 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) - scan store_sales + partial aggregation over (ss_sold_date_sk, ss_store_sk) + dynamic filter (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) @@ -40,10 +41,11 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_day_name_85, d_week_seq_75, ss_store_sk_52) 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) + intermediate 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) - scan store_sales + partial aggregation over (ss_sold_date_sk_68, ss_store_sk_52) + dynamic filter (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) 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 fef17d23291c..2a6e60377a62 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 @@ -8,18 +8,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_6) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_6]) - partial aggregation over (i_item_id_6) + intermediate aggregation over (i_item_id_6) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_item_sk) join (INNER, REPLICATED): - dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales + join (INNER, REPLICATED): + dynamic filter (ss_addr_sk::EQUAL, ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -36,18 +37,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_83) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_83]) - partial aggregation over (i_item_id_83) + intermediate aggregation over (i_item_id_83) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_item_sk) join (INNER, REPLICATED): - dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) - scan catalog_sales + join (INNER, REPLICATED): + dynamic filter (cs_bill_addr_sk::EQUAL, cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): @@ -64,18 +66,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_183) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_183]) - partial aggregation over (i_item_id_183) + intermediate aggregation over (i_item_id_183) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ws_item_sk) join (INNER, REPLICATED): - dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) - scan web_sales + join (INNER, REPLICATED): + dynamic filter (ws_bill_addr_sk::EQUAL, ws_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_address + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): 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 de56116567d9..36f27668b336 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 @@ -5,16 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (gid, sm_type, web_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [gid, sm_type, web_name]) - partial aggregation over (gid, sm_type, web_name) + intermediate aggregation over (gid, sm_type, web_name) join (INNER, REPLICATED): 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_ship_mode_sk, ws_warehouse_sk, ws_web_site_sk) + join (INNER, REPLICATED): + dynamic filter (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, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan web_site 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 98123fc17ccb..b69e1d6abb2d 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 @@ -6,95 +6,97 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk]) - partial aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + intermediate aggregation over (ca_city, ca_city_105, ca_street_name, ca_street_name_102, ca_street_number, ca_street_number_101, ca_zip, ca_zip_108, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_addr_sk]) - join (INNER, REPLICATED): + intermediate aggregation over (c_current_addr_sk, ca_city, ca_street_name, ca_street_number, ca_zip, d_year, d_year_15, d_year_45, i_product_name, s_store_name, s_zip, ss_item_sk) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_addr_sk]) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (c_current_addr_sk, d_year, d_year_15, d_year_45, s_store_name, s_zip, ss_addr_sk, ss_item_sk) join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_customer_sk]) join (INNER, REPLICATED): join (INNER, REPLICATED): - 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) - scan store_sales + join (INNER, REPLICATED): + join (INNER, REPLICATED): + 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) + 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) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [cs_item_sk]) + 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) + 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) + scan catalog_returns 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) - scan store_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [cs_item_sk]) - 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) - 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) - scan catalog_returns + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + dynamic filter (s_store_name::EQUAL, s_zip::EQUAL) + scan store local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (s_store_name::EQUAL, s_zip::EQUAL) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + remote exchange (REPARTITION, HASH, [c_customer_sk]) join (INNER, REPLICATED): - 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) - scan customer + join (INNER, REPLICATED): + join (INNER, REPLICATED): + 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) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + dynamic filter (hd_income_band_sk::EQUAL) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (hd_income_band_sk::EQUAL) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + join (INNER, REPLICATED): + dynamic filter (hd_income_band_sk_91::EQUAL) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (hd_income_band_sk_91::EQUAL) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - dynamic filter (i_item_sk::EQUAL) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [ca_address_sk]) - scan customer_address + dynamic filter (i_item_sk::EQUAL) + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [ca_address_sk]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_99]) scan customer_address @@ -103,93 +105,95 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (ca_city_445, ca_city_460, ca_street_name_442, ca_street_name_457, ca_street_number_441, ca_street_number_456, ca_zip_448, ca_zip_463, d_year_247, d_year_277, d_year_307, i_product_name_500, s_store_name_336, s_zip_356, ss_item_sk_127) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_city_445, ca_city_460, ca_street_name_442, ca_street_name_457, ca_street_number_441, ca_street_number_456, ca_zip_448, ca_zip_463, d_year_247, d_year_277, d_year_307, i_product_name_500, s_store_name_336, s_zip_356, ss_item_sk_127]) - partial aggregation over (ca_city_445, ca_city_460, ca_street_name_442, ca_street_name_457, ca_street_number_441, ca_street_number_456, ca_zip_448, ca_zip_463, d_year_247, d_year_277, d_year_307, i_product_name_500, s_store_name_336, s_zip_356, ss_item_sk_127) + intermediate aggregation over (ca_city_445, ca_city_460, ca_street_name_442, ca_street_name_457, ca_street_number_441, ca_street_number_456, ca_zip_448, ca_zip_463, d_year_247, d_year_277, d_year_307, i_product_name_500, s_store_name_336, s_zip_356, ss_item_sk_127) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [c_current_addr_sk_366]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_addr_sk_131]) - join (INNER, REPLICATED): + intermediate aggregation over (c_current_addr_sk_366, ca_city_445, ca_street_name_442, ca_street_number_441, ca_zip_448, d_year_247, d_year_277, d_year_307, i_product_name_500, s_store_name_336, s_zip_356, ss_item_sk_127) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_addr_sk_131]) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (c_current_addr_sk_366, d_year_247, d_year_277, d_year_307, s_store_name_336, s_zip_356, ss_addr_sk_131, ss_item_sk_127) join (INNER, REPLICATED): - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_customer_sk_128]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, REPLICATED): + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_customer_sk_128]) join (INNER, REPLICATED): join (INNER, REPLICATED): - 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) - scan store_sales + join (INNER, REPLICATED): + join (INNER, REPLICATED): + 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) + 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) + scan store_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (cs_item_sk_187) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [cs_item_sk_187]) + 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) + 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) + scan catalog_returns 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) - scan store_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (cs_item_sk_187) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [cs_item_sk_187]) - 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) - 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) - scan catalog_returns + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics + scan store local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_customer_sk_362]) - join (INNER, REPLICATED): - join (INNER, REPLICATED): + remote exchange (REPARTITION, HASH, [c_customer_sk_362]) join (INNER, REPLICATED): - 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) - scan customer + join (INNER, REPLICATED): + join (INNER, REPLICATED): + 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) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan promotion + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + dynamic filter (hd_income_band_sk_426::EQUAL) + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (hd_income_band_sk_426::EQUAL) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band + join (INNER, REPLICATED): + dynamic filter (hd_income_band_sk_433::EQUAL) + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan income_band local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (hd_income_band_sk_433::EQUAL) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan income_band - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [ca_address_sk_439]) - scan customer_address + scan item + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [ca_address_sk_439]) + scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_454]) scan customer_address 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 f280b4666b6a..8d466dcc8369 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 @@ -12,22 +12,23 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1]) - partial aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) + intermediate aggregation over (d_moy, d_year, w_city_2, w_country_5, w_county_3, w_state_4, w_warehouse_name_0, w_warehouse_sq_ft_1) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (d_moy, d_year, ws_warehouse_sk) 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) - scan web_sales + join (INNER, REPLICATED): + dynamic filter (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, []) + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse @@ -39,22 +40,23 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy_87, d_year_85, w_city_71, w_country_75, w_county_72, w_state_73, w_warehouse_name_65, w_warehouse_sq_ft_66) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy_87, d_year_85, w_city_71, w_country_75, w_county_72, w_state_73, w_warehouse_name_65, w_warehouse_sq_ft_66]) - partial aggregation over (d_moy_87, d_year_85, w_city_71, w_country_75, w_county_72, w_state_73, w_warehouse_name_65, w_warehouse_sq_ft_66) + intermediate aggregation over (d_moy_87, d_year_85, w_city_71, w_country_75, w_county_72, w_state_73, w_warehouse_name_65, w_warehouse_sq_ft_66) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_warehouse_sk, d_moy_87, d_year_85) 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) - scan catalog_sales + join (INNER, REPLICATED): + dynamic filter (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, []) + scan ship_mode local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan ship_mode + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan time_dim + scan time_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan warehouse 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 2e5533e17682..07612f6d9750 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 @@ -16,24 +16,25 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) final aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) local exchange (GATHER, SINGLE, []) - partial aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) + intermediate aggregation over (ca_address_sk, ca_city, ss_customer_sk, ss_ticket_number) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ca_address_sk]) dynamic filter (ca_address_sk::EQUAL) scan customer_address local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_addr_sk]) - join (INNER, REPLICATED): + partial aggregation over (ss_addr_sk, ss_customer_sk, ss_ticket_number) join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) - scan store_sales + join (INNER, REPLICATED): + dynamic filter (ss_hdemo_sk::EQUAL, 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, []) - scan date_dim + scan store local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan store - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan household_demographics 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 12803f1108ad..6ab7f3cf3bf1 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 @@ -26,14 +26,15 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_state_53) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_state_53]) - partial aggregation over (s_state_53) + intermediate aggregation over (s_state_53) join (INNER, REPLICATED): - join (INNER, REPLICATED): - dynamic filter (ss_sold_date_sk_26::EQUAL, ss_store_sk_10::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_store_sk_10) + join (INNER, REPLICATED): + dynamic filter (ss_sold_date_sk_26::EQUAL, ss_store_sk_10::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store 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..b30519203776 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 @@ -4,28 +4,31 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (i_brand, i_brand_id, t_hour, t_minute) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_brand, i_brand_id, t_hour, t_minute]) - partial aggregation over (i_brand, i_brand_id, t_hour, t_minute) + intermediate aggregation over (i_brand, i_brand_id, t_hour, t_minute) join (INNER, REPLICATED): 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) - 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) - 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ws_item_sk, ws_sold_time_sk) + join (INNER, REPLICATED): + dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL, ws_sold_time_sk::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + partial aggregation over (cs_item_sk, cs_sold_time_sk) + join (INNER, REPLICATED): + dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL, cs_sold_time_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim + partial aggregation over (ss_item_sk, ss_sold_time_sk) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_sold_time_sk::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 6efa7856761d..a21ad70d1e44 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 @@ -7,15 +7,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_customer_id_65, c_first_name_72, c_last_name_73, d_year_115) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_65]) - partial aggregation over (c_customer_id_65, c_first_name_72, c_last_name_73, d_year_115) + intermediate aggregation over (c_customer_id_65, c_first_name_72, c_last_name_73, d_year_115) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk_86]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk_86::EQUAL, ss_sold_date_sk_106::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_year_115, ss_customer_sk_86) + join (INNER, REPLICATED): + partial aggregation over (ss_customer_sk_86, ss_sold_date_sk_106) + dynamic filter (ss_customer_sk_86::EQUAL, ss_sold_date_sk_106::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_64]) dynamic filter (c_customer_id_65::EQUAL, c_customer_id_65::EQUAL) @@ -23,15 +25,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_customer_id_313, c_first_name_320, c_last_name_321, d_year_374) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_313]) - partial aggregation over (c_customer_id_313, c_first_name_320, c_last_name_321, d_year_374) + intermediate aggregation over (c_customer_id_313, c_first_name_320, c_last_name_321, d_year_374) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_335]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_335::EQUAL, ws_sold_date_sk_365::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_year_374, ws_bill_customer_sk_335) + join (INNER, REPLICATED): + partial aggregation over (ws_bill_customer_sk_335, ws_sold_date_sk_365) + dynamic filter (ws_bill_customer_sk_335::EQUAL, ws_sold_date_sk_365::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_312]) dynamic filter (c_customer_id_313::EQUAL) @@ -40,15 +44,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_customer_id, c_first_name, c_last_name, d_year) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id]) - partial aggregation over (c_customer_id, c_first_name, c_last_name, d_year) + intermediate aggregation over (c_customer_id, c_first_name, c_last_name, d_year) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ss_customer_sk]) - join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_year, ss_customer_sk) + join (INNER, REPLICATED): + partial aggregation over (ss_customer_sk, ss_sold_date_sk) + dynamic filter (ss_customer_sk::EQUAL, 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, [c_customer_sk]) dynamic filter (c_customer_id::EQUAL) @@ -56,15 +62,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (c_customer_id_484, c_first_name_491, c_last_name_492, d_year_545) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_id_484]) - partial aggregation over (c_customer_id_484, c_first_name_491, c_last_name_492, d_year_545) + intermediate aggregation over (c_customer_id_484, c_first_name_491, c_last_name_492, d_year_545) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [ws_bill_customer_sk_506]) - join (INNER, REPLICATED): - dynamic filter (ws_bill_customer_sk_506::EQUAL, ws_sold_date_sk_536::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_year_545, ws_bill_customer_sk_506) + join (INNER, REPLICATED): + partial aggregation over (ws_bill_customer_sk_506, ws_sold_date_sk_536) + dynamic filter (ws_bill_customer_sk_506::EQUAL, ws_sold_date_sk_536::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_483]) scan customer 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 b6012ced7baf..04316fa58c35 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 @@ -5,11 +5,12 @@ local exchange (GATHER, SINGLE, []) final aggregation over (channel, col_name, d_qoy, d_year, i_category) local exchange (REPARTITION, HASH, [channel, col_name, d_qoy, d_year, i_category]) remote exchange (REPARTITION, HASH, [d_qoy_10, d_year_9, expr_134, expr_135, i_category_6]) - partial aggregation over (d_qoy_10, d_year_9, expr_134, expr_135, i_category_6) + intermediate 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) - scan store_sales + partial aggregation over (expr_134, expr_135, ss_item_sk, ss_sold_date_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item @@ -17,25 +18,27 @@ local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim remote exchange (REPARTITION, HASH, [d_qoy_51, d_year_47, expr_140, expr_141, i_category_29]) - partial aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) + intermediate aggregation over (d_qoy_51, d_year_47, expr_140, expr_141, i_category_29) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_sold_date_sk]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_item_sk]) - dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [i_item_sk_17]) - scan item + partial aggregation over (expr_140, expr_141, i_category_29, ws_sold_date_sk) + remote exchange (REPARTITION, HASH, [ws_sold_date_sk]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ws_item_sk]) + dynamic filter (ws_item_sk::EQUAL, ws_sold_date_sk::EQUAL) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [i_item_sk_17]) + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_date_sk_41]) scan date_dim remote exchange (REPARTITION, HASH, [d_qoy_111, d_year_107, expr_131, expr_133, i_category_89]) - partial aggregation over (d_qoy_111, d_year_107, expr_131, expr_133, i_category_89) + intermediate 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) - scan catalog_sales + partial aggregation over (cs_item_sk, cs_sold_date_sk, expr_131, expr_133) + dynamic filter (cs_item_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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..56f631c95ed1 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 @@ -11,31 +11,31 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ss_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_store_sk]) - partial aggregation over (ss_store_sk) - join (INNER, REPLICATED): + join (INNER, REPLICATED): + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): dynamic filter (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, []) - scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store final aggregation over (sr_store_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [sr_store_sk]) - partial aggregation over (sr_store_sk) - join (INNER, REPLICATED): + join (INNER, REPLICATED): + partial aggregation over (sr_store_sk) join (INNER, REPLICATED): dynamic filter (sr_returned_date_sk::EQUAL, sr_store_sk::EQUAL) scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan store + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan store cross join: final aggregation over (cs_call_center_sk) local exchange (GATHER, SINGLE, []) @@ -63,28 +63,28 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ws_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_web_page_sk]) - partial aggregation over (ws_web_page_sk) - join (INNER, REPLICATED): + join (INNER, REPLICATED): + partial aggregation over (ws_web_page_sk) join (INNER, REPLICATED): dynamic filter (ws_sold_date_sk::EQUAL, ws_web_page_sk::EQUAL) scan web_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page final aggregation over (wr_web_page_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [wr_web_page_sk]) - partial aggregation over (wr_web_page_sk) - join (INNER, REPLICATED): + join (INNER, REPLICATED): + partial aggregation over (wr_web_page_sk) join (INNER, REPLICATED): dynamic filter (wr_returned_date_sk::EQUAL, wr_web_page_sk::EQUAL) scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan web_page + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan web_page 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 dc7f02ee0adc..ff5c9cf58f60 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 @@ -6,16 +6,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year_16, ws_bill_customer_sk, ws_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ws_bill_customer_sk, ws_item_sk]) - partial aggregation over (d_year_16, ws_bill_customer_sk, ws_item_sk) + intermediate aggregation over (d_year_16, ws_bill_customer_sk, ws_item_sk) join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) - dynamic filter (ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_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) - scan web_returns + partial aggregation over (ws_bill_customer_sk, ws_item_sk, ws_sold_date_sk) + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, [ws_item_sk, ws_order_number]) + dynamic filter (ws_bill_customer_sk::EQUAL, ws_item_sk::EQUAL, ws_sold_date_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) + scan web_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -25,15 +26,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_year, ss_customer_sk, ss_item_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_year, ss_customer_sk, ss_item_sk]) - partial aggregation over (d_year, ss_customer_sk, ss_item_sk) + intermediate aggregation over (d_year, ss_customer_sk, ss_item_sk) join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) - dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - scan store_returns + partial aggregation over (ss_customer_sk, ss_item_sk, ss_sold_date_sk) + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, [ss_item_sk, ss_ticket_number]) + dynamic filter (ss_customer_sk::EQUAL, ss_sold_date_sk::EQUAL) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim @@ -42,15 +44,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_54) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cs_bill_customer_sk, cs_item_sk, d_year_54]) - partial aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_54) + intermediate aggregation over (cs_bill_customer_sk, cs_item_sk, d_year_54) join (INNER, REPLICATED): - join (LEFT, PARTITIONED): - remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) - dynamic filter (cs_sold_date_sk::EQUAL) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - scan catalog_returns + partial aggregation over (cs_bill_customer_sk, cs_item_sk, cs_sold_date_sk) + join (LEFT, PARTITIONED): + remote exchange (REPARTITION, HASH, [cs_item_sk, cs_order_number]) + dynamic filter (cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) + scan catalog_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan date_dim 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 05ec5b1e9b58..d66c9d0ea3d5 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 @@ -6,18 +6,19 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ss_customer_sk]) - partial aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) + intermediate aggregation over (s_city, ss_addr_sk, ss_customer_sk, ss_ticket_number) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_addr_sk, ss_customer_sk, ss_store_sk, ss_ticket_number) join (INNER, REPLICATED): - dynamic filter (ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) - scan store_sales + join (INNER, REPLICATED): + dynamic filter (ss_customer_sk::EQUAL, ss_hdemo_sk::EQUAL, 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, []) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics + scan household_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store 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 5aee30f1d60f..accbe68f9967 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 @@ -10,84 +10,87 @@ local exchange (GATHER, SINGLE, []) final aggregation over (s_store_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [s_store_id]) - partial aggregation over (s_store_id) + intermediate aggregation over (s_store_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ss_store_sk) join (INNER, REPLICATED): 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) - scan store_sales + 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) + scan store_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) + dynamic filter (sr_item_sk::EQUAL) + scan store_returns local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [sr_item_sk, sr_ticket_number]) - dynamic filter (sr_item_sk::EQUAL) - scan store_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan store final aggregation over (cp_catalog_page_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cp_catalog_page_id]) - partial aggregation over (cp_catalog_page_id) + intermediate aggregation over (cp_catalog_page_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cs_catalog_page_sk) join (INNER, REPLICATED): 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) - scan catalog_sales + 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) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) + dynamic filter (cr_item_sk::EQUAL) + scan catalog_returns local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [cr_item_sk, cr_order_number]) - dynamic filter (cr_item_sk::EQUAL) - scan catalog_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan catalog_page final aggregation over (web_site_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [web_site_id]) - partial aggregation over (web_site_id) + intermediate aggregation over (web_site_id) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (ws_web_site_sk) join (INNER, REPLICATED): 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) - scan web_sales + 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) + scan web_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) + dynamic filter (wr_item_sk::EQUAL) + scan web_returns local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [wr_item_sk, wr_order_number]) - dynamic filter (wr_item_sk::EQUAL) - scan web_returns + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan promotion local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan promotion - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan item + scan item local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan web_site 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 d83f55c49136..543b501656de 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 @@ -8,15 +8,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state, cr_returning_customer_sk) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_state, cr_returning_customer_sk]) - partial aggregation over (ca_state, cr_returning_customer_sk) + intermediate aggregation over (ca_state, cr_returning_customer_sk) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cr_returning_addr_sk]) - join (INNER, REPLICATED): - dynamic filter (cr_returned_date_sk::EQUAL, cr_returning_addr_sk::EQUAL, cr_returning_customer_sk::EQUAL) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_returning_addr_sk, cr_returning_customer_sk) + join (INNER, REPLICATED): + dynamic filter (cr_returned_date_sk::EQUAL, cr_returning_addr_sk::EQUAL, cr_returning_customer_sk::EQUAL) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk]) scan customer_address @@ -37,15 +38,16 @@ local exchange (GATHER, SINGLE, []) final aggregation over (ca_state_88, cr_returning_customer_sk_27) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_state_88, cr_returning_customer_sk_27]) - partial aggregation over (ca_state_88, cr_returning_customer_sk_27) + intermediate aggregation over (ca_state_88, cr_returning_customer_sk_27) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [cr_returning_addr_sk_30]) - join (INNER, REPLICATED): - dynamic filter (cr_returned_date_sk_47::EQUAL, cr_returning_addr_sk_30::EQUAL) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cr_returning_addr_sk_30, cr_returning_customer_sk_27) + join (INNER, REPLICATED): + dynamic filter (cr_returned_date_sk_47::EQUAL, cr_returning_addr_sk_30::EQUAL) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [ca_address_sk_80]) scan customer_address 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 574aadb02221..3241bbd32d61 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 @@ -6,68 +6,70 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id]) - partial aggregation over (i_item_id) + intermediate aggregation over (i_item_id) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [sr_item_sk]) - join (INNER, REPLICATED): - dynamic filter (sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) - scan store_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (d_date::EQUAL) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - 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) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_40) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_week_seq_40]) - partial aggregation over (d_week_seq_40) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [i_item_sk]) - dynamic filter (i_item_id::EQUAL) - scan item - join (INNER, PARTITIONED): - final aggregation over (i_item_id_74) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [i_item_id_74]) - partial aggregation over (i_item_id_74) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [cr_item_sk]) + partial aggregation over (sr_item_sk) + remote exchange (REPARTITION, HASH, [sr_item_sk]) join (INNER, REPLICATED): - dynamic filter (cr_item_sk::EQUAL, cr_returned_date_sk::EQUAL) - scan catalog_returns + dynamic filter (sr_item_sk::EQUAL, sr_returned_date_sk::EQUAL) + scan store_returns local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) join (INNER, REPLICATED): - dynamic filter (d_date_99::EQUAL) + dynamic filter (d_date::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_date_131) + final aggregation over (d_date_6) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_date_131]) - partial aggregation over (d_date_131) + 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_133::EQUAL) + dynamic filter (d_week_seq_8::EQUAL) scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_165) + final aggregation over (d_week_seq_40) local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_week_seq_165]) - partial aggregation over (d_week_seq_165) + remote exchange (REPARTITION, HASH, [d_week_seq_40]) + partial aggregation over (d_week_seq_40) scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [i_item_sk]) + dynamic filter (i_item_id::EQUAL) + scan item + join (INNER, PARTITIONED): + final aggregation over (i_item_id_74) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [i_item_id_74]) + intermediate aggregation over (i_item_id_74) + join (INNER, PARTITIONED): + partial aggregation over (cr_item_sk) + remote exchange (REPARTITION, HASH, [cr_item_sk]) + join (INNER, REPLICATED): + dynamic filter (cr_item_sk::EQUAL, cr_returned_date_sk::EQUAL) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + dynamic filter (d_date_99::EQUAL) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + 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) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_165) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [d_week_seq_165]) + partial aggregation over (d_week_seq_165) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_sk_73]) dynamic filter (i_item_id_74::EQUAL) @@ -75,33 +77,34 @@ local exchange (GATHER, SINGLE, []) final aggregation over (i_item_id_200) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_id_200]) - partial aggregation over (i_item_id_200) + intermediate aggregation over (i_item_id_200) join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [wr_item_sk]) - join (INNER, REPLICATED): - dynamic filter (wr_item_sk::EQUAL, wr_returned_date_sk::EQUAL) - scan web_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, REPLICATED): - dynamic filter (d_date_225::EQUAL) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - 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) - scan date_dim - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - final aggregation over (d_week_seq_291) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [d_week_seq_291]) - partial aggregation over (d_week_seq_291) - scan date_dim + partial aggregation over (wr_item_sk) + remote exchange (REPARTITION, HASH, [wr_item_sk]) + join (INNER, REPLICATED): + dynamic filter (wr_item_sk::EQUAL, wr_returned_date_sk::EQUAL) + scan web_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, REPLICATED): + dynamic filter (d_date_225::EQUAL) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + 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) + scan date_dim + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + final aggregation over (d_week_seq_291) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [d_week_seq_291]) + partial aggregation over (d_week_seq_291) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_item_sk_199]) scan item 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..66612900fd6c 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 @@ -8,15 +8,17 @@ final aggregation over () final aggregation over (c_first_name_6, c_last_name_7, d_date_3) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_6, c_last_name_7, d_date_3]) - partial aggregation over (c_first_name_6, c_last_name_7, d_date_3) + intermediate aggregation over (c_first_name_6, c_last_name_7, d_date_3) 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_date_3, ss_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (ss_customer_sk, ss_sold_date_sk) + dynamic filter (ss_customer_sk::EQUAL, 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, [c_customer_sk]) scan customer @@ -24,15 +26,17 @@ final aggregation over () final aggregation over (c_first_name_50, c_last_name_51, d_date_14) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_50, c_last_name_51, d_date_14]) - partial aggregation over (c_first_name_50, c_last_name_51, d_date_14) + intermediate aggregation over (c_first_name_50, c_last_name_51, d_date_14) 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (cs_bill_customer_sk, d_date_14) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (cs_bill_customer_sk, cs_sold_date_sk) + dynamic filter (cs_bill_customer_sk::EQUAL, cs_sold_date_sk::EQUAL) + scan catalog_sales + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_customer_sk_42]) scan customer @@ -40,15 +44,17 @@ final aggregation over () final aggregation over (c_first_name_102, c_last_name_103, d_date_66) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [c_first_name_102, c_last_name_103, d_date_66]) - partial aggregation over (c_first_name_102, c_last_name_103, d_date_66) + intermediate aggregation over (c_first_name_102, c_last_name_103, d_date_66) 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) - scan web_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + intermediate aggregation over (d_date_66, ws_bill_customer_sk) + join (INNER, REPLICATED, can skip output duplicates): + partial aggregation over (ws_bill_customer_sk, ws_sold_date_sk) + dynamic filter (ws_bill_customer_sk::EQUAL, 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, [c_customer_sk_94]) scan customer 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..25539de540e1 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 @@ -7,12 +7,13 @@ local exchange (GATHER, SINGLE, []) final aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [d_moy, i_brand, i_category, i_class, s_company_name, s_store_name]) - partial aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) + intermediate aggregation over (d_moy, i_brand, i_category, i_class, s_company_name, s_store_name) join (INNER, REPLICATED): join (INNER, REPLICATED): join (INNER, REPLICATED): - dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) - scan store_sales + partial aggregation over (ss_item_sk, ss_sold_date_sk, ss_store_sk) + dynamic filter (ss_item_sk::EQUAL, ss_sold_date_sk::EQUAL, ss_store_sk::EQUAL) + scan store_sales local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan item 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 f1f04de0ccd3..db3fba55c484 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 @@ -4,33 +4,34 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status]) - partial aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) + intermediate aggregation over (cc_call_center_id, cc_manager, cc_name, cd_education_status, cd_marital_status) join (INNER, REPLICATED): - join (INNER, REPLICATED): + partial aggregation over (cd_education_status, cd_marital_status, cr_call_center_sk) join (INNER, REPLICATED): - dynamic filter (cr_call_center_sk::EQUAL, cr_returned_date_sk::EQUAL, cr_returning_customer_sk::EQUAL) - scan catalog_returns - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [ca_address_sk]) - dynamic filter (ca_address_sk::EQUAL) - scan customer_address - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [c_current_addr_sk]) - join (INNER, REPLICATED): + join (INNER, REPLICATED): + dynamic filter (cr_call_center_sk::EQUAL, cr_returned_date_sk::EQUAL, cr_returning_customer_sk::EQUAL) + scan catalog_returns + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [ca_address_sk]) + dynamic filter (ca_address_sk::EQUAL) + scan customer_address + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [c_current_addr_sk]) join (INNER, REPLICATED): - dynamic filter (c_current_cdemo_sk::EQUAL, c_current_hdemo_sk::EQUAL) - scan customer + join (INNER, REPLICATED): + dynamic filter (c_current_cdemo_sk::EQUAL, c_current_hdemo_sk::EQUAL) + scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan customer_demographics local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) - scan customer_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan household_demographics - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + scan household_demographics + local exchange (GATHER, SINGLE, []) + remote exchange (REPLICATE, BROADCAST, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center 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 423d9b2be09f..ab8a1d92eea9 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 @@ -6,14 +6,15 @@ remote exchange (GATHER, SINGLE, []) final aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [i_category, i_class, i_current_price, i_item_desc, i_item_id]) - partial aggregation over (i_category, i_class, i_current_price, i_item_desc, i_item_id) + intermediate 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) - scan store_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (ss_item_sk) + join (INNER, REPLICATED): + dynamic filter (ss_item_sk::EQUAL, 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 (REPLICATE, BROADCAST, []) scan item 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 208b2e372ec0..85d482617bca 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 @@ -5,16 +5,17 @@ local exchange (GATHER, SINGLE, []) final aggregation over (cc_name, gid, sm_type) local exchange (GATHER, SINGLE, []) remote exchange (REPARTITION, HASH, [cc_name, gid, sm_type]) - partial aggregation over (cc_name, gid, sm_type) + intermediate aggregation over (cc_name, gid, sm_type) join (INNER, REPLICATED): 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) - scan catalog_sales - local exchange (GATHER, SINGLE, []) - remote exchange (REPLICATE, BROADCAST, []) - scan date_dim + partial aggregation over (cs_call_center_sk, cs_ship_mode_sk, cs_warehouse_sk) + join (INNER, REPLICATED): + dynamic filter (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, []) + scan date_dim local exchange (GATHER, SINGLE, []) remote exchange (REPLICATE, BROADCAST, []) scan call_center 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 5186d3df8fa7..54a50eaff701 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 @@ -4,17 +4,17 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, ROUND_ROBIN, []) final aggregation over (orderdate, orderkey_4, shippriority) local exchange (GATHER, SINGLE, []) - partial aggregation over (orderdate, orderkey_4, shippriority) - join (INNER, PARTITIONED): + join (INNER, PARTITIONED): + partial aggregation over (orderkey_4) remote exchange (REPARTITION, HASH, [orderkey_4]) dynamic filter (orderkey_4::EQUAL) scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [orderkey]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [custkey_0]) - dynamic filter (custkey_0::EQUAL) - scan orders - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [custkey]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [orderkey]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [custkey_0]) + dynamic filter (custkey_0::EQUAL) + scan orders + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [custkey]) + scan customer 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 a3fe7724a42f..593117d58e0c 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 @@ -4,24 +4,24 @@ local exchange (GATHER, SINGLE, []) local exchange (REPARTITION, ROUND_ROBIN, []) final aggregation over (custkey_0, name, orderdate, orderkey_4, totalprice) local exchange (GATHER, SINGLE, []) - partial aggregation over (custkey_0, name, orderdate, orderkey_4, totalprice) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [orderkey_4]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [orderkey_4]) + partial aggregation over (orderkey_4) dynamic filter (orderkey_4::EQUAL) scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [orderkey]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [custkey_0]) - join (INNER, PARTITIONED): - remote exchange (REPARTITION, HASH, [orderkey]) - dynamic filter (custkey_0::EQUAL, orderkey::EQUAL) - scan orders - final aggregation over (orderkey_8) - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [orderkey_8]) - partial aggregation over (orderkey_8) - scan lineitem - local exchange (GATHER, SINGLE, []) - remote exchange (REPARTITION, HASH, [custkey]) - scan customer + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [orderkey]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [custkey_0]) + join (INNER, PARTITIONED): + remote exchange (REPARTITION, HASH, [orderkey]) + dynamic filter (custkey_0::EQUAL, orderkey::EQUAL) + scan orders + final aggregation over (orderkey_8) + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [orderkey_8]) + partial aggregation over (orderkey_8) + scan lineitem + local exchange (GATHER, SINGLE, []) + remote exchange (REPARTITION, HASH, [custkey]) + scan customer 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 504261454dd2..ff74d8be0a1a 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 @@ -15,11 +15,12 @@ local exchange (GATHER, SINGLE, []) scan lineitem final aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) local exchange (GATHER, SINGLE, []) - partial aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) + intermediate aggregation over (commitdate, name, name_9, nationkey, orderkey_13, orderstatus, receiptdate, suppkey_0, unique_54) join (INNER, PARTITIONED, can skip output duplicates): remote exchange (REPARTITION, HASH, [orderkey_13]) - dynamic filter (orderkey_13::EQUAL) - scan lineitem + partial aggregation over (orderkey_13, suppkey_15) + dynamic filter (orderkey_13::EQUAL) + scan lineitem local exchange (GATHER, SINGLE, []) join (INNER, PARTITIONED): remote exchange (REPARTITION, HASH, [orderkey])