From 9685294cdfa9c2ae0822413c36d72ca495004204 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Thu, 28 Nov 2024 09:54:16 +0800 Subject: [PATCH 01/17] [opt](nereids) support pushdown agg distinct through join --- .../doris/nereids/jobs/executor/Rewriter.java | 2 + .../apache/doris/nereids/rules/RuleType.java | 1 + ...DownAggWithDistinctThroughJoinOneSide.java | 175 ++++++++++++++++++ .../trees/plans/logical/LogicalAggregate.java | 6 + 4 files changed, 184 insertions(+) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index fa00a4e9343325..dbaaf2a6b32244 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -113,6 +113,7 @@ import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoinOnPkFk; import org.apache.doris.nereids.rules.rewrite.PushDownAggThroughJoinOneSide; +import org.apache.doris.nereids.rules.rewrite.PushDownAggWithDistinctThroughJoinOneSide; import org.apache.doris.nereids.rules.rewrite.PushDownDistinctThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughProject; import org.apache.doris.nereids.rules.rewrite.PushDownLimit; @@ -344,6 +345,7 @@ public class Rewriter extends AbstractBatchJobExecutor { topic("Eager aggregation", costBased(topDown( + new PushDownAggWithDistinctThroughJoinOneSide(), new PushDownAggThroughJoinOneSide(), new PushDownAggThroughJoin() )), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index c81f56f85b6add..e25050b012dc5f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -201,6 +201,7 @@ public enum RuleType { ELIMINATE_SORT(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE(RuleTypeClass.REWRITE), + PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN(RuleTypeClass.REWRITE), PUSH_DOWN_AGG_THROUGH_JOIN_ON_PKFK(RuleTypeClass.REWRITE), TRANSPOSE_LOGICAL_SEMI_JOIN_LOGICAL_JOIN(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java new file mode 100644 index 00000000000000..94778347fa963e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -0,0 +1,175 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; +import org.apache.doris.nereids.trees.expressions.functions.agg.Max; +import org.apache.doris.nereids.trees.expressions.functions.agg.Min; +import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Push down agg function with distinct through join on only one side. + */ +public class PushDownAggWithDistinctThroughJoinOneSide implements RewriteRuleFactory { + @Override + public List buildRules() { + return ImmutableList.of( + logicalAggregate(logicalProject(innerLogicalJoin())) + .when(agg -> agg.child().isAllSlots()) + .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) + .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) + .when(agg -> { + Set funcs = agg.getAggregateFunctions(); + return !funcs.isEmpty() && funcs.stream() + .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum + || (f instanceof Count && (!((Count) f).isCountStar()))) && f.isDistinct() + && f.child(0) instanceof Slot); + }) + .thenApply(ctx -> { + LogicalAggregate>> agg = ctx.root; + return pushDownAggWithDistinct(agg, agg.child().child(), agg.child().getProjects()); + }) + .toRule(RuleType.PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE) + ); + } + + private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate agg, + LogicalJoin join, List projects) { + List leftOutput = join.left().getOutput(); + List rightOutput = join.right().getOutput(); + + List leftFuncs = new ArrayList<>(); + List rightFuncs = new ArrayList<>(); + for (AggregateFunction func : agg.getAggregateFunctions()) { + Slot slot = (Slot) func.child(0); + if (leftOutput.contains(slot)) { + leftFuncs.add(func); + } else if (rightOutput.contains(slot)) { + rightFuncs.add(func); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } + if (leftFuncs.isEmpty() && rightFuncs.isEmpty()) { + return null; + } + + Set leftGroupBy = new HashSet<>(); + Set rightGroupBy = new HashSet<>(); + for (Expression e : agg.getGroupByExpressions()) { + Slot slot = (Slot) e; + if (leftOutput.contains(slot)) { + leftGroupBy.add(slot); + } else if (rightOutput.contains(slot)) { + rightGroupBy.add(slot); + } else { + return null; + } + } + join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { + if (leftOutput.contains(slot)) { + leftGroupBy.add(slot); + } else if (rightOutput.contains(slot)) { + rightGroupBy.add(slot); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + })); + + Plan left = join.left(); + Plan right = join.right(); + Map leftSlotToOutput = new HashMap<>(); + Map rightSlotToOutput = new HashMap<>(); + if (!leftFuncs.isEmpty()) { + Builder leftAggOutputBuilder = ImmutableList.builder() + .addAll(leftGroupBy); + leftFuncs.forEach(func -> { + Alias alias = func.alias(func.getName()); + leftSlotToOutput.put((Slot) func.child(0), alias); + }); + left = new LogicalAggregate<>(ImmutableList.copyOf(leftGroupBy), leftAggOutputBuilder.build(), join.left()); + } + if (!rightFuncs.isEmpty()) { + Builder rightAggOutputBuilder = ImmutableList.builder() + .addAll(rightGroupBy); + rightFuncs.forEach(func -> { + Alias alias = func.alias(func.getName()); + rightSlotToOutput.put((Slot) func.child(0), alias); + rightAggOutputBuilder.add(alias); + }); + right = new LogicalAggregate<>(ImmutableList.copyOf(rightGroupBy), rightAggOutputBuilder.build(), + join.right()); + } + + Preconditions.checkState(left != join.left() || right != join.right()); + Plan newJoin = join.withChildren(left, right); + + LogicalAggregate midAgg = agg.withChildGroupByAndOutput( + ImmutableList.copyOf(leftGroupBy), projects, newJoin); + + List newOutputExprs = new ArrayList<>(); + for (NamedExpression ne : agg.getOutputExpressions()) { + if (ne instanceof Alias && ((Alias) ne).child() instanceof AggregateFunction) { + AggregateFunction func = (AggregateFunction) ((Alias) ne).child(); + Slot slot = (Slot) func.child(0); + if (leftSlotToOutput.containsKey(slot)) { + Expression newFunc = discardDistinct(func); + newOutputExprs.add((NamedExpression) ne.withChildren(newFunc)); + } else if (rightSlotToOutput.containsKey(slot)) { + Expression newFunc = discardDistinct(func); + newOutputExprs.add((NamedExpression) ne.withChildren(newFunc)); + } else { + throw new IllegalStateException("Slot " + slot + " not found in join output"); + } + } else { + newOutputExprs.add(ne); + } + } + return agg.withAggOutputChild(newOutputExprs, midAgg); + } + + private static Expression discardDistinct(AggregateFunction func) { + Preconditions.checkState(func.isDistinct(), "current aggregation function is not distinct"); + Set aggChild = Sets.newLinkedHashSet(func.children()); + AggregateFunction nonDistinct = func.withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); + return nonDistinct; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java index 2798b1aef0102b..a06f238f5314ad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java @@ -271,6 +271,12 @@ public LogicalAggregate withGroupByAndOutput(List groupByExprL hasPushed, sourceRepeat, Optional.empty(), Optional.empty(), child()); } + public LogicalAggregate withChildGroupByAndOutput(List groupByExprList, + List outputExpressionList, Plan newChild) { + return new LogicalAggregate<>(groupByExprList, outputExpressionList, normalized, ordinalIsResolved, generated, + hasPushed, sourceRepeat, Optional.empty(), Optional.empty(), newChild); + } + public LogicalAggregate withChildAndOutput(CHILD_TYPE child, List outputExpressionList) { return new LogicalAggregate<>(groupByExpressions, outputExpressionList, normalized, ordinalIsResolved, From cc33fd64a470bd7f00db59411bc7a480390e0566 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Thu, 28 Nov 2024 12:13:51 +0800 Subject: [PATCH 02/17] [opt](nereids) support pushdown agg distinct through join --- ...DownAggWithDistinctThroughJoinOneSide.java | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java index 94778347fa963e..f5ff03d5411cde 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -58,9 +58,9 @@ public List buildRules() { .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) .when(agg -> { Set funcs = agg.getAggregateFunctions(); - return !funcs.isEmpty() && funcs.stream() + return funcs.size() == 1 && funcs.stream() .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || (f instanceof Count && (!((Count) f).isCountStar()))) && f.isDistinct() + || f instanceof Count) && f.isDistinct() && f.child(0) instanceof Slot); }) .thenApply(ctx -> { @@ -78,8 +78,10 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate leftFuncs = new ArrayList<>(); List rightFuncs = new ArrayList<>(); + Set middleAggGroupBy = new HashSet<>(); for (AggregateFunction func : agg.getAggregateFunctions()) { Slot slot = (Slot) func.child(0); + middleAggGroupBy.add(slot); if (leftOutput.contains(slot)) { leftFuncs.add(func); } else if (rightOutput.contains(slot)) { @@ -88,7 +90,9 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate pushDownAggWithDistinct(LogicalAggregate rightGroupBy = new HashSet<>(); for (Expression e : agg.getGroupByExpressions()) { Slot slot = (Slot) e; + middleAggGroupBy.add(slot); if (leftOutput.contains(slot)) { leftGroupBy.add(slot); } else if (rightOutput.contains(slot)) { @@ -113,12 +118,12 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate leftSlotToOutput = new HashMap<>(); Map rightSlotToOutput = new HashMap<>(); - if (!leftFuncs.isEmpty()) { + if (isLeftSideAggDistinct) { + leftGroupBy.add((Slot) leftFuncs.get(0).child(0)); Builder leftAggOutputBuilder = ImmutableList.builder() .addAll(leftGroupBy); leftFuncs.forEach(func -> { @@ -126,8 +131,8 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate(ImmutableList.copyOf(leftGroupBy), leftAggOutputBuilder.build(), join.left()); - } - if (!rightFuncs.isEmpty()) { + } else { + rightGroupBy.add((Slot) rightFuncs.get(0).child(0)); Builder rightAggOutputBuilder = ImmutableList.builder() .addAll(rightGroupBy); rightFuncs.forEach(func -> { @@ -142,8 +147,8 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate midAgg = agg.withChildGroupByAndOutput( - ImmutableList.copyOf(leftGroupBy), projects, newJoin); + LogicalAggregate middleAgg = agg.withChildGroupByAndOutput( + ImmutableList.copyOf(middleAggGroupBy), projects, newJoin); List newOutputExprs = new ArrayList<>(); for (NamedExpression ne : agg.getOutputExpressions()) { @@ -163,7 +168,7 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate Date: Thu, 28 Nov 2024 15:10:03 +0800 Subject: [PATCH 03/17] [opt](nereids) support pushdown agg distinct through join --- ...DownAggWithDistinctThroughJoinOneSide.java | 86 ++++++++++--------- 1 file changed, 46 insertions(+), 40 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java index f5ff03d5411cde..02f6c2b825fbd6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -55,13 +55,19 @@ public List buildRules() { logicalAggregate(logicalProject(innerLogicalJoin())) .when(agg -> agg.child().isAllSlots()) .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) + .when(agg -> !agg.isGenerated()) + .whenNot(agg -> agg.getAggregateFunctions().isEmpty()) .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) .when(agg -> { Set funcs = agg.getAggregateFunctions(); - return funcs.size() == 1 && funcs.stream() - .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum - || f instanceof Count) && f.isDistinct() - && f.child(0) instanceof Slot); + if (funcs.size() > 1) { + return false; + } else { + return funcs.stream() + .allMatch(f -> (f instanceof Min || f instanceof Max || f instanceof Sum + || f instanceof Count) && f.isDistinct() + && f.child(0) instanceof Slot); + } }) .thenApply(ctx -> { LogicalAggregate>> agg = ctx.root; @@ -73,18 +79,20 @@ public List buildRules() { private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate agg, LogicalJoin join, List projects) { - List leftOutput = join.left().getOutput(); - List rightOutput = join.right().getOutput(); + Plan leftJoin = join.left(); + Plan rightJoin = join.right(); + List leftJoinOutput = leftJoin.getOutput(); + List rightJoinOutput = rightJoin.getOutput(); List leftFuncs = new ArrayList<>(); List rightFuncs = new ArrayList<>(); - Set middleAggGroupBy = new HashSet<>(); + Set newAggOverJoinGroupByKeys = new HashSet<>(); for (AggregateFunction func : agg.getAggregateFunctions()) { Slot slot = (Slot) func.child(0); - middleAggGroupBy.add(slot); - if (leftOutput.contains(slot)) { + newAggOverJoinGroupByKeys.add(slot); + if (leftJoinOutput.contains(slot)) { leftFuncs.add(func); - } else if (rightOutput.contains(slot)) { + } else if (rightJoinOutput.contains(slot)) { rightFuncs.add(func); } else { throw new IllegalStateException("Slot " + slot + " not found in join output"); @@ -96,59 +104,58 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate leftGroupBy = new HashSet<>(); - Set rightGroupBy = new HashSet<>(); + Set leftPushDownGroupBy = new HashSet<>(); + Set rightPushDownGroupBy = new HashSet<>(); for (Expression e : agg.getGroupByExpressions()) { Slot slot = (Slot) e; - middleAggGroupBy.add(slot); - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); + newAggOverJoinGroupByKeys.add(slot); + if (leftJoinOutput.contains(slot)) { + leftPushDownGroupBy.add(slot); + } else if (rightJoinOutput.contains(slot)) { + rightPushDownGroupBy.add(slot); } else { return null; } } join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> { - if (leftOutput.contains(slot)) { - leftGroupBy.add(slot); - } else if (rightOutput.contains(slot)) { - rightGroupBy.add(slot); + if (leftJoinOutput.contains(slot)) { + leftPushDownGroupBy.add(slot); + } else if (rightJoinOutput.contains(slot)) { + rightPushDownGroupBy.add(slot); } else { throw new IllegalStateException("Slot " + slot + " not found in join output"); } })); - Plan left = join.left(); - Plan right = join.right(); + Map leftSlotToOutput = new HashMap<>(); Map rightSlotToOutput = new HashMap<>(); if (isLeftSideAggDistinct) { - leftGroupBy.add((Slot) leftFuncs.get(0).child(0)); + leftPushDownGroupBy.add((Slot) leftFuncs.get(0).child(0)); Builder leftAggOutputBuilder = ImmutableList.builder() - .addAll(leftGroupBy); + .addAll(leftPushDownGroupBy); leftFuncs.forEach(func -> { - Alias alias = func.alias(func.getName()); + Alias alias = func.alias("PDADT_" + func.getName()); leftSlotToOutput.put((Slot) func.child(0), alias); }); - left = new LogicalAggregate<>(ImmutableList.copyOf(leftGroupBy), leftAggOutputBuilder.build(), join.left()); + leftJoin = new LogicalAggregate<>(ImmutableList.copyOf(leftPushDownGroupBy), + leftAggOutputBuilder.build(), join.left()); } else { - rightGroupBy.add((Slot) rightFuncs.get(0).child(0)); + rightPushDownGroupBy.add((Slot) rightFuncs.get(0).child(0)); Builder rightAggOutputBuilder = ImmutableList.builder() - .addAll(rightGroupBy); + .addAll(rightPushDownGroupBy); rightFuncs.forEach(func -> { - Alias alias = func.alias(func.getName()); + Alias alias = func.alias("PDADT_" + func.getName()); rightSlotToOutput.put((Slot) func.child(0), alias); rightAggOutputBuilder.add(alias); }); - right = new LogicalAggregate<>(ImmutableList.copyOf(rightGroupBy), rightAggOutputBuilder.build(), - join.right()); + rightJoin = new LogicalAggregate<>(ImmutableList.copyOf(rightPushDownGroupBy), + rightAggOutputBuilder.build(), join.right()); } - Preconditions.checkState(left != join.left() || right != join.right()); - Plan newJoin = join.withChildren(left, right); - - LogicalAggregate middleAgg = agg.withChildGroupByAndOutput( - ImmutableList.copyOf(middleAggGroupBy), projects, newJoin); + Preconditions.checkState(leftJoin != join.left() || rightJoin != join.right()); + Plan newJoin = join.withChildren(leftJoin, rightJoin); + LogicalAggregate newAggOverJoin = agg.withChildGroupByAndOutput( + ImmutableList.copyOf(newAggOverJoinGroupByKeys), projects, newJoin); List newOutputExprs = new ArrayList<>(); for (NamedExpression ne : agg.getOutputExpressions()) { @@ -168,13 +175,12 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate aggChild = Sets.newLinkedHashSet(func.children()); - AggregateFunction nonDistinct = func.withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); - return nonDistinct; + return func.withDistinctAndChildren(false, ImmutableList.copyOf(aggChild)); } } From d8aae1dc9ac0af09aaabe6169b1bfdbca92d5685 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Thu, 28 Nov 2024 20:49:33 +0800 Subject: [PATCH 04/17] [opt](nereids) support pushdown agg distinct through join --- ...aggr_distinct_through_join_one_side.groovy | 259 ++++++++++++++++++ 1 file changed, 259 insertions(+) create mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..e1b31c82672b13 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy @@ -0,0 +1,259 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_aggr_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + + sql """ + DROP TABLE IF EXISTS aggr_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into aggr_with_distinct_t values (1, 1, 'a')" + sql "insert into aggr_with_distinct_t values (2, null, 'a')" + sql "insert into aggr_with_distinct_t values (3, 1, null)" + sql "insert into aggr_with_distinct_t values (4, 2, 'b')" + sql "insert into aggr_with_distinct_t values (5, null, 'b')" + sql "insert into aggr_with_distinct_t values (6, 2, null)" + sql "insert into aggr_with_distinct_t values (7, 3, 'c')" + sql "insert into aggr_with_distinct_t values (8, null, 'c')" + sql "insert into aggr_with_distinct_t values (9, 3, null)" + sql "insert into aggr_with_distinct_t values (10, null, null)" + sql "analyze table aggr_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select min(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select min(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_function_processed_columns """ + select min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} From 0ed5e3f6ecf2514fe74eb2e9ab661ec90633e308 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Thu, 28 Nov 2024 21:33:56 +0800 Subject: [PATCH 05/17] [opt](nereids) support pushdown agg distinct through join --- ...wn_aggr_distinct_through_join_one_side.out | 237 ++++++++++++++++ ...n_count_distinct_through_join_one_side.out | 237 ++++++++++++++++ ...own_min_distinct_through_join_one_side.out | 237 ++++++++++++++++ ...own_sum_distinct_through_join_one_side.out | 231 ++++++++++++++++ ...aggr_distinct_through_join_one_side.groovy | 259 ------------------ ...distinct_through_join_one_side_cust.groovy | 126 +++++++++ ...ount_distinct_through_join_one_side.groovy | 259 ++++++++++++++++++ ..._min_distinct_through_join_one_side.groovy | 259 ++++++++++++++++++ ..._sum_distinct_through_join_one_side.groovy | 255 +++++++++++++++++ 9 files changed, 1841 insertions(+), 259 deletions(-) create mode 100644 regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out create mode 100644 regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out create mode 100644 regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out create mode 100644 regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out delete mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy create mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy create mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy create mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy create mode 100644 regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..db15483c496eb4 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out @@ -0,0 +1,237 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..4a84ff5117c008 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out @@ -0,0 +1,237 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_right_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_full_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_semi_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +1 3.0 +3 2.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +1 +1 +3 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +1 2 +1 3 +3 6 + +-- !groupby_pushdown_multi_table_join -- +1 +1 +1 + +-- !groupby_pushdown_with_order_by -- +1 +1 +1 +3 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +1 +1 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 1 +3 1 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 1 +c 1 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 1 1 +c 1 1 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 1.5 1 +1 4.5 1 +1 7.5 1 +3 7.0 0 + +-- !groupby_pushdown_with_order_by_limit -- +1 +1 +1 +3 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +1 +1 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +0 +1 +1 +1 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_right_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_full_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +1 3.0 +3 2.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +1 2 +1 3 +3 6 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 1 +3 1 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 1 +c 1 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 1 1 +c 1 1 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 1.5 1 +1 4.5 1 +1 7.5 1 +3 7.0 0 + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +1 +1 +3 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +1 +1 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_function_processed_columns -- +0 +1 +1 +1 + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..db15483c496eb4 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.out @@ -0,0 +1,237 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_right_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_full_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +1 2.0 +2 2.0 +3 3.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +1 6 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 1.5 a +1 7.0 \N +2 4.5 b +3 7.5 c + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +1 +2 +3 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_function_processed_columns -- +\N +1 +1 +1 + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out new file mode 100644 index 00000000000000..bb8366176a7ba6 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.out @@ -0,0 +1,231 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !groupby_pushdown_basic -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_right_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_full_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_semi_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_left_anti_join -- + +-- !groupby_pushdown_complex_conditions -- + +-- !groupby_pushdown_with_aggregate -- +1 1.0 +2 2.0 +3 3.0 +6 2.0 + +-- !groupby_pushdown_subquery -- + +-- !groupby_pushdown_outer_join -- +1 +2 +3 +6 + +-- !groupby_pushdown_deep_subquery -- + +-- !groupby_pushdown_having -- + +-- !groupby_pushdown_mixed_aggregates -- +1 1 +2 2 +3 3 +6 6 + +-- !groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !groupby_pushdown_with_order_by -- +1 +2 +3 +6 + +-- !groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !groupby_pushdown_with_where_clause -- + +-- !groupby_pushdown_varied_aggregates -- +1 +2 +3 +6 + +-- !groupby_pushdown_with_order_by_limit -- +1 +2 +3 +6 + +-- !groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !groupby_pushdown_complex_join_condition -- + +-- !groupby_pushdown_function_processed_columns -- +1 +2 +3 +6 + +-- !groupby_pushdown_nested_queries -- + +-- !with_hint_groupby_pushdown_basic -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_right_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_full_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_semi_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_left_anti_join -- + +-- !with_hint_groupby_pushdown_complex_conditions -- + +-- !with_hint_groupby_pushdown_with_aggregate -- +1 1.0 +2 2.0 +3 3.0 +6 2.0 + +-- !with_hint_groupby_pushdown_subquery -- + +-- !with_hint_groupby_pushdown_outer_join -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_deep_subquery -- + +-- !with_hint_groupby_pushdown_having -- + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +1 1 +2 2 +3 3 +6 6 + +-- !with_hint_groupby_pushdown_multi_table_join -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_with_order_by -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +1 1 +2 2 +3 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- +a 1 +b 2 +c 3 + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- +a 1 1 +b 2 2 +c 3 3 + +-- !with_hint_groupby_pushdown_with_where_clause -- + +-- !with_hint_groupby_pushdown_varied_aggregates -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +1 +2 +3 +6 + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +1 +2 +3 + +-- !with_hint_groupby_pushdown_complex_join_condition -- + +-- !with_hint_groupby_pushdown_nested_queries -- + diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy deleted file mode 100644 index e1b31c82672b13..00000000000000 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.groovy +++ /dev/null @@ -1,259 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -suite("push_down_aggr_distinct_through_join_one_side") { - sql "SET enable_nereids_planner=true" - sql "set runtime_filter_mode=OFF" - sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" - - sql """ - DROP TABLE IF EXISTS aggr_with_distinct_t; - """ - - sql """ - CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( - `id` int(32), - `score` int(64) NULL, - `name` varchar(64) NULL - ) ENGINE = OLAP - DISTRIBUTED BY HASH(id) BUCKETS 4 - PROPERTIES ( - "replication_allocation" = "tag.location.default: 1" - ); - """ - - sql "insert into aggr_with_distinct_t values (1, 1, 'a')" - sql "insert into aggr_with_distinct_t values (2, null, 'a')" - sql "insert into aggr_with_distinct_t values (3, 1, null)" - sql "insert into aggr_with_distinct_t values (4, 2, 'b')" - sql "insert into aggr_with_distinct_t values (5, null, 'b')" - sql "insert into aggr_with_distinct_t values (6, 2, null)" - sql "insert into aggr_with_distinct_t values (7, 3, 'c')" - sql "insert into aggr_with_distinct_t values (8, null, 'c')" - sql "insert into aggr_with_distinct_t values (9, 3, null)" - sql "insert into aggr_with_distinct_t values (10, null, null)" - sql "analyze table aggr_with_distinct_t with sync;" - order_qt_groupby_pushdown_basic """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_left_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_right_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_full_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_left_semi_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_left_anti_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_complex_conditions """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_with_aggregate """ - select min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_subquery """ - select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_outer_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_deep_subquery """ - select min(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_having """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; - """ - - order_qt_groupby_pushdown_mixed_aggregates """ - select min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_multi_table_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; - """ - - order_qt_groupby_pushdown_with_order_by """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; - """ - - order_qt_groupby_pushdown_multiple_equal_conditions """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_equal_conditions_with_aggregate """ - select sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_equal_conditions_non_aggregate """ - select t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_with_where_clause """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; - """ - - order_qt_groupby_pushdown_varied_aggregates """ - select min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_with_order_by_limit """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; - """ - - order_qt_groupby_pushdown_alias_multiple_equal_conditions """ - select min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; - """ - - order_qt_groupby_pushdown_complex_join_condition """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; - """ - - order_qt_groupby_pushdown_function_processed_columns """ - select min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; - """ - - order_qt_groupby_pushdown_nested_queries """ - select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_basic """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_left_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_right_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_full_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_left_semi_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_left_anti_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_complex_conditions """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_with_aggregate """ - select min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_subquery """ - select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_outer_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_deep_subquery """ - select min(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_having """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; - """ - - order_qt_with_hint_groupby_pushdown_mixed_aggregates """ - select min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_multi_table_join """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_with_order_by """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ - select sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ - select t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_with_where_clause """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_varied_aggregates """ - select min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_with_order_by_limit """ - select min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; - """ - - order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ - select min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; - """ - - order_qt_with_hint_groupby_pushdown_complex_join_condition """ - select min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_function_processed_columns """ - select min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; - """ - - order_qt_with_hint_groupby_pushdown_nested_queries """ - select min(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; - """ -} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy new file mode 100644 index 00000000000000..6101c4bf9f537b --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy @@ -0,0 +1,126 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_aggr_distinct_through_join_one_side_cust") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" + + sql """ + DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymd; + DROP TABLE IF EXISTS dwd_tracking_sensor_init_tmp_ymd; + """ + + sql """ + CREATE TABLE `dwd_com_abtest_result_inc_ymd` ( + `app_name` varchar(255) NULL, + `user_key` text NULL, + `group_name` text NULL, + `dt` date NOT NULL, + ) ENGINE=OLAP + DUPLICATE KEY(`app_name`) + AUTO PARTITION BY RANGE (date_trunc(`dt`, 'day')) + (PARTITION p20240813000000 VALUES [('2024-08-13'), ('2024-08-14')), + PARTITION p20240814000000 VALUES [('2024-08-14'), ('2024-08-15')), + PARTITION p20240815000000 VALUES [('2024-08-15'), ('2024-08-16')), + PARTITION p20240816000000 VALUES [('2024-08-16'), ('2024-08-17')), + PARTITION p20240817000000 VALUES [('2024-08-17'), ('2024-08-18')), + PARTITION p20240818000000 VALUES [('2024-08-18'), ('2024-08-19')), + PARTITION p20240819000000 VALUES [('2024-08-19'), ('2024-08-20'))) + DISTRIBUTED BY HASH(`app_name`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728" + ); + + CREATE TABLE `dwd_tracking_sensor_init_tmp_ymd` ( + `ip` varchar(20) NULL, + `gz_user_id` text NULL, + `dt` date NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`ip`) + AUTO PARTITION BY RANGE (date_trunc(`dt`, 'day')) + (PARTITION p20240813000000 VALUES [('2024-08-13'), ('2024-08-14')), + PARTITION p20240814000000 VALUES [('2024-08-14'), ('2024-08-15')), + PARTITION p20240815000000 VALUES [('2024-08-15'), ('2024-08-16')), + PARTITION p20240816000000 VALUES [('2024-08-16'), ('2024-08-17')), + PARTITION p20240817000000 VALUES [('2024-08-17'), ('2024-08-18')), + PARTITION p20240818000000 VALUES [('2024-08-18'), ('2024-08-19')), + PARTITION p20240819000000 VALUES [('2024-08-19'), ('2024-08-20'))) + DISTRIBUTED BY HASH(`ip`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "min_load_replica_num" = "-1", + "is_being_synced" = "false", + "storage_medium" = "hdd", + "storage_format" = "V2", + "inverted_index_storage_format" = "V2", + "light_schema_change" = "true", + "disable_auto_compaction" = "false", + "enable_single_replica_compaction" = "false", + "group_commit_interval_ms" = "10000", + "group_commit_data_bytes" = "134217728" + ); + """ + + explain { + sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymd.gz_user_id) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymd.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymd " + + "LEFT JOIN dwd_com_abtest_result_inc_ymd " + + "ON dwd_tracking_sensor_init_tmp_ymd.gz_user_id = dwd_com_abtest_result_inc_ymd.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymd.dt = dwd_com_abtest_result_inc_ymd.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); + contains"groupByExpr=[gz_user_id#1, dt#2]" + contains"groupByExpr=[gz_user_id#1, dt#2, group_name#5], outputExpr=[gz_user_id#1, dt#2, group_name#5]" + contains"[group_name#5, dt#2]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, count(partial_count(gz_user_id)#13) AS `a2c1a830_1`#7]" + } + + explain { + sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymd.ip) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymd.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymd " + + "LEFT JOIN dwd_com_abtest_result_inc_ymd " + + "ON dwd_tracking_sensor_init_tmp_ymd.gz_user_id = dwd_com_abtest_result_inc_ymd.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymd.dt = dwd_com_abtest_result_inc_ymd.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); + contains"groupByExpr=[ip#0, gz_user_id#1, dt#2], outputExpr=[ip#0, gz_user_id#1, dt#2]" + contains"groupByExpr=[ip#0, dt#2, group_name#5], outputExpr=[ip#0, dt#2, group_name#5]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, partial_count(ip#0) AS `partial_count(ip)`#13]" + contains"groupByExpr=[group_name#5, dt#2], outputExpr=[group_name#5, dt#2, count(partial_count(ip)#13) AS `a2c1a830_1`#7]" + } +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..57b932e9fbdbb2 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -0,0 +1,259 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_count_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + + sql """ + DROP TABLE IF EXISTS aggr_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into aggr_with_distinct_t values (1, 1, 'a')" + sql "insert into aggr_with_distinct_t values (2, null, 'a')" + sql "insert into aggr_with_distinct_t values (3, 1, null)" + sql "insert into aggr_with_distinct_t values (4, 2, 'b')" + sql "insert into aggr_with_distinct_t values (5, null, 'b')" + sql "insert into aggr_with_distinct_t values (6, 2, null)" + sql "insert into aggr_with_distinct_t values (7, 3, 'c')" + sql "insert into aggr_with_distinct_t values (8, null, 'c')" + sql "insert into aggr_with_distinct_t values (9, 3, null)" + sql "insert into aggr_with_distinct_t values (10, null, null)" + sql "analyze table aggr_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..e9214b38fd86e5 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy @@ -0,0 +1,259 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_min_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + + sql """ + DROP TABLE IF EXISTS aggr_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into aggr_with_distinct_t values (1, 1, 'a')" + sql "insert into aggr_with_distinct_t values (2, null, 'a')" + sql "insert into aggr_with_distinct_t values (3, 1, null)" + sql "insert into aggr_with_distinct_t values (4, 2, 'b')" + sql "insert into aggr_with_distinct_t values (5, null, 'b')" + sql "insert into aggr_with_distinct_t values (6, 2, null)" + sql "insert into aggr_with_distinct_t values (7, 3, 'c')" + sql "insert into aggr_with_distinct_t values (8, null, 'c')" + sql "insert into aggr_with_distinct_t values (9, 3, null)" + sql "insert into aggr_with_distinct_t values (10, null, null)" + sql "analyze table aggr_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_function_processed_columns """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy new file mode 100644 index 00000000000000..6afe8acbb0be6a --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy @@ -0,0 +1,255 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("push_down_sum_distinct_through_join_one_side") { + sql "SET enable_nereids_planner=true" + sql "set runtime_filter_mode=OFF" + sql "SET enable_fallback_to_original_planner=false" + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + + sql """ + DROP TABLE IF EXISTS aggr_with_distinct_t; + """ + + sql """ + CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + `id` int(32), + `score` int(64) NULL, + `name` varchar(64) NULL + ) ENGINE = OLAP + DISTRIBUTED BY HASH(id) BUCKETS 4 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql "insert into aggr_with_distinct_t values (1, 1, 'a')" + sql "insert into aggr_with_distinct_t values (2, null, 'a')" + sql "insert into aggr_with_distinct_t values (3, 1, null)" + sql "insert into aggr_with_distinct_t values (4, 2, 'b')" + sql "insert into aggr_with_distinct_t values (5, null, 'b')" + sql "insert into aggr_with_distinct_t values (6, 2, null)" + sql "insert into aggr_with_distinct_t values (7, 3, 'c')" + sql "insert into aggr_with_distinct_t values (8, null, 'c')" + sql "insert into aggr_with_distinct_t values (9, 3, null)" + sql "insert into aggr_with_distinct_t values (10, null, null)" + sql "analyze table aggr_with_distinct_t with sync;" + order_qt_groupby_pushdown_basic """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_right_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_full_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_semi_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_left_anti_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_complex_conditions """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_aggregate """ + select sum(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_subquery """ + select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_outer_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_deep_subquery """ + select sum(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_having """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + """ + + order_qt_groupby_pushdown_mixed_aggregates """ + select sum(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_multi_table_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_groupby_pushdown_multiple_equal_conditions """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_with_where_clause """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_groupby_pushdown_varied_aggregates """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_with_order_by_limit """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + """ + + order_qt_groupby_pushdown_alias_multiple_equal_conditions """ + select sum(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_groupby_pushdown_complex_join_condition """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_groupby_pushdown_function_processed_columns """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_groupby_pushdown_nested_queries """ + select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_basic """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_right_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_full_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_semi_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_left_anti_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_conditions """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_aggregate """ + select sum(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_subquery """ + select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_outer_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_deep_subquery """ + select sum(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_having """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + """ + + order_qt_with_hint_groupby_pushdown_mixed_aggregates """ + select sum(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multi_table_join """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + select sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ + select t1.name, sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_where_clause """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_varied_aggregates """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_with_order_by_limit """ + select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + """ + + order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + select sum(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + order_qt_with_hint_groupby_pushdown_complex_join_condition """ + select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + order_qt_with_hint_groupby_pushdown_nested_queries """ + select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} From 3fed78c8170b02fd72d3595d17c4d2a0ac24304c Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Thu, 28 Nov 2024 21:36:01 +0800 Subject: [PATCH 06/17] [opt](nereids) support pushdown agg distinct through join --- ...wn_aggr_distinct_through_join_one_side.out | 237 ------------------ 1 file changed, 237 deletions(-) delete mode 100644 regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out deleted file mode 100644 index db15483c496eb4..00000000000000 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side.out +++ /dev/null @@ -1,237 +0,0 @@ --- This file is automatically generated. You should know what you did if you want to edit this --- !groupby_pushdown_basic -- -1 -1 -2 -3 - --- !groupby_pushdown_left_join -- -1 -1 -2 -3 - --- !groupby_pushdown_right_join -- -1 -1 -2 -3 - --- !groupby_pushdown_full_join -- -1 -1 -2 -3 - --- !groupby_pushdown_left_semi_join -- -1 -1 -2 -3 - --- !groupby_pushdown_left_anti_join -- - --- !groupby_pushdown_complex_conditions -- - --- !groupby_pushdown_with_aggregate -- -1 1.0 -1 2.0 -2 2.0 -3 3.0 - --- !groupby_pushdown_subquery -- - --- !groupby_pushdown_outer_join -- -1 -1 -2 -3 - --- !groupby_pushdown_deep_subquery -- - --- !groupby_pushdown_having -- - --- !groupby_pushdown_mixed_aggregates -- -1 1 -1 6 -2 2 -3 3 - --- !groupby_pushdown_multi_table_join -- -1 -2 -3 - --- !groupby_pushdown_with_order_by -- -1 -1 -2 -3 - --- !groupby_pushdown_multiple_equal_conditions -- -1 -2 -3 - --- !groupby_pushdown_equal_conditions_with_aggregate -- -1 1 -2 2 -3 3 - --- !groupby_pushdown_equal_conditions_non_aggregate -- -a 1 -b 2 -c 3 - --- !groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- -a 1 1 -b 2 2 -c 3 3 - --- !groupby_pushdown_with_where_clause -- - --- !groupby_pushdown_varied_aggregates -- -1 1.5 a -1 7.0 \N -2 4.5 b -3 7.5 c - --- !groupby_pushdown_with_order_by_limit -- -1 -1 -2 -3 - --- !groupby_pushdown_alias_multiple_equal_conditions -- -1 -2 -3 - --- !groupby_pushdown_complex_join_condition -- - --- !groupby_pushdown_function_processed_columns -- -\N -1 -1 -1 - --- !groupby_pushdown_nested_queries -- - --- !with_hint_groupby_pushdown_basic -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_left_join -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_right_join -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_full_join -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_left_semi_join -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_left_anti_join -- - --- !with_hint_groupby_pushdown_complex_conditions -- - --- !with_hint_groupby_pushdown_with_aggregate -- -1 1.0 -1 2.0 -2 2.0 -3 3.0 - --- !with_hint_groupby_pushdown_subquery -- - --- !with_hint_groupby_pushdown_outer_join -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_deep_subquery -- - --- !with_hint_groupby_pushdown_having -- - --- !with_hint_groupby_pushdown_mixed_aggregates -- -1 1 -1 6 -2 2 -3 3 - --- !with_hint_groupby_pushdown_multi_table_join -- -1 -2 -3 - --- !with_hint_groupby_pushdown_with_order_by -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_multiple_equal_conditions -- -1 -2 -3 - --- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- -1 1 -2 2 -3 3 - --- !with_hint_groupby_pushdown_equal_conditions_non_aggregate -- -a 1 -b 2 -c 3 - --- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate -- -a 1 1 -b 2 2 -c 3 3 - --- !with_hint_groupby_pushdown_with_where_clause -- - --- !with_hint_groupby_pushdown_varied_aggregates -- -1 1.5 a -1 7.0 \N -2 4.5 b -3 7.5 c - --- !with_hint_groupby_pushdown_with_order_by_limit -- -1 -1 -2 -3 - --- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- -1 -2 -3 - --- !with_hint_groupby_pushdown_complex_join_condition -- - --- !with_hint_groupby_pushdown_function_processed_columns -- -\N -1 -1 -1 - --- !with_hint_groupby_pushdown_nested_queries -- - From 2b9e6ad696fa15307bd1a38e7f9dc124632b716b Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Fri, 29 Nov 2024 10:40:16 +0800 Subject: [PATCH 07/17] [opt](nereids) support pushdown agg distinct through join --- ...ount_distinct_through_join_one_side.groovy | 130 +++++++++--------- ..._min_distinct_through_join_one_side.groovy | 130 +++++++++--------- ..._sum_distinct_through_join_one_side.groovy | 128 ++++++++--------- 3 files changed, 194 insertions(+), 194 deletions(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index 57b932e9fbdbb2..c9e276e50f5001 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -23,11 +23,11 @@ suite("push_down_count_distinct_through_join_one_side") { sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ - DROP TABLE IF EXISTS aggr_with_distinct_t; + DROP TABLE IF EXISTS count_with_distinct_t; """ sql """ - CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + CREATE TABLE IF NOT EXISTS count_with_distinct_t( `id` int(32), `score` int(64) NULL, `name` varchar(64) NULL @@ -38,222 +38,222 @@ suite("push_down_count_distinct_through_join_one_side") { ); """ - sql "insert into aggr_with_distinct_t values (1, 1, 'a')" - sql "insert into aggr_with_distinct_t values (2, null, 'a')" - sql "insert into aggr_with_distinct_t values (3, 1, null)" - sql "insert into aggr_with_distinct_t values (4, 2, 'b')" - sql "insert into aggr_with_distinct_t values (5, null, 'b')" - sql "insert into aggr_with_distinct_t values (6, 2, null)" - sql "insert into aggr_with_distinct_t values (7, 3, 'c')" - sql "insert into aggr_with_distinct_t values (8, null, 'c')" - sql "insert into aggr_with_distinct_t values (9, 3, null)" - sql "insert into aggr_with_distinct_t values (10, null, null)" - sql "analyze table aggr_with_distinct_t with sync;" + sql "insert into count_with_distinct_t values (1, 1, 'a')" + sql "insert into count_with_distinct_t values (2, null, 'a')" + sql "insert into count_with_distinct_t values (3, 1, null)" + sql "insert into count_with_distinct_t values (4, 2, 'b')" + sql "insert into count_with_distinct_t values (5, null, 'b')" + sql "insert into count_with_distinct_t values (6, 2, null)" + sql "insert into count_with_distinct_t values (7, 3, 'c')" + sql "insert into count_with_distinct_t values (8, null, 'c')" + sql "insert into count_with_distinct_t values (9, 3, null)" + sql "insert into count_with_distinct_t values (10, null, null)" + sql "analyze table count_with_distinct_t with sync;" order_qt_groupby_pushdown_basic """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_right_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 right join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_full_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 full join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_semi_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 inner join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_anti_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left anti join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_complex_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_groupby_pushdown_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_outer_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_deep_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t) count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_having """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; """ order_qt_groupby_pushdown_mixed_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_multi_table_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id join count_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_groupby_pushdown_with_order_by """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_groupby_pushdown_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_with_where_clause """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_groupby_pushdown_varied_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_with_order_by_limit """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; """ order_qt_groupby_pushdown_alias_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from count_with_distinct_t t1_alias join count_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_groupby_pushdown_complex_join_condition """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_groupby_pushdown_function_processed_columns """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_nested_queries """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_basic """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_right_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 right join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_full_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 full join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_semi_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 inner join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_anti_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left anti join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_complex_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_outer_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 left join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_deep_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t) count_with_distinct_t where score > 10) t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_having """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name having count(distinct t1.score) > 100; """ order_qt_with_hint_groupby_pushdown_mixed_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), sum(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_multi_table_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id join count_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, count(distinct t1.score), count(distinct t2.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_where_clause """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_with_hint_groupby_pushdown_varied_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score), avg(t1.id), count(distinct t2.name) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by_limit """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name order by count(distinct t1.score) limit 10; """ order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1_alias.score) from count_with_distinct_t t1_alias join count_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_with_hint_groupby_pushdown_complex_join_condition """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1 join count_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_function_processed_columns """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct LENGTH(t1.name)) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_nested_queries """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from count_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy index e9214b38fd86e5..f47a019f0dd33c 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy @@ -23,11 +23,11 @@ suite("push_down_min_distinct_through_join_one_side") { sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ - DROP TABLE IF EXISTS aggr_with_distinct_t; + DROP TABLE IF EXISTS min_with_distinct_t; """ sql """ - CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + CREATE TABLE IF NOT EXISTS min_with_distinct_t( `id` int(32), `score` int(64) NULL, `name` varchar(64) NULL @@ -38,222 +38,222 @@ suite("push_down_min_distinct_through_join_one_side") { ); """ - sql "insert into aggr_with_distinct_t values (1, 1, 'a')" - sql "insert into aggr_with_distinct_t values (2, null, 'a')" - sql "insert into aggr_with_distinct_t values (3, 1, null)" - sql "insert into aggr_with_distinct_t values (4, 2, 'b')" - sql "insert into aggr_with_distinct_t values (5, null, 'b')" - sql "insert into aggr_with_distinct_t values (6, 2, null)" - sql "insert into aggr_with_distinct_t values (7, 3, 'c')" - sql "insert into aggr_with_distinct_t values (8, null, 'c')" - sql "insert into aggr_with_distinct_t values (9, 3, null)" - sql "insert into aggr_with_distinct_t values (10, null, null)" - sql "analyze table aggr_with_distinct_t with sync;" + sql "insert into min_with_distinct_t values (1, 1, 'a')" + sql "insert into min_with_distinct_t values (2, null, 'a')" + sql "insert into min_with_distinct_t values (3, 1, null)" + sql "insert into min_with_distinct_t values (4, 2, 'b')" + sql "insert into min_with_distinct_t values (5, null, 'b')" + sql "insert into min_with_distinct_t values (6, 2, null)" + sql "insert into min_with_distinct_t values (7, 3, 'c')" + sql "insert into min_with_distinct_t values (8, null, 'c')" + sql "insert into min_with_distinct_t values (9, 3, null)" + sql "insert into min_with_distinct_t values (10, null, null)" + sql "analyze table min_with_distinct_t with sync;" order_qt_groupby_pushdown_basic """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_right_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 right join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_full_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 full join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_semi_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 inner join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_anti_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left anti join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_complex_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_groupby_pushdown_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_outer_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_deep_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t) min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_having """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; """ order_qt_groupby_pushdown_mixed_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_multi_table_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id join min_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_groupby_pushdown_with_order_by """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_groupby_pushdown_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_with_where_clause """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_groupby_pushdown_varied_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_with_order_by_limit """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; """ order_qt_groupby_pushdown_alias_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from min_with_distinct_t t1_alias join min_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_groupby_pushdown_complex_join_condition """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_groupby_pushdown_function_processed_columns """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_nested_queries """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_basic """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_right_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 right join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_full_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 full join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_semi_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 inner join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_anti_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left anti join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_complex_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_outer_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 left join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_deep_subquery """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t) min_with_distinct_t where score > 10) t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_having """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name having min(distinct t1.score) > 100; """ order_qt_with_hint_groupby_pushdown_mixed_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), sum(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_multi_table_join """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id join min_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ sum(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ t1.name, min(distinct t1.score), min(distinct t2.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_where_clause """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_with_hint_groupby_pushdown_varied_aggregates """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score), avg(t1.id), min(distinct t2.name) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by_limit """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name order by min(distinct t1.score) limit 10; """ order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1_alias.score) from min_with_distinct_t t1_alias join min_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_with_hint_groupby_pushdown_complex_join_condition """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from min_with_distinct_t t1 join min_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_function_processed_columns """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct LENGTH(t1.name)) from min_with_distinct_t t1, min_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_nested_queries """ - select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ min(distinct t1.score) from (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where score > 20) t1 join (select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ * from min_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy index 6afe8acbb0be6a..708323eec4d731 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy @@ -23,11 +23,11 @@ suite("push_down_sum_distinct_through_join_one_side") { sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" sql """ - DROP TABLE IF EXISTS aggr_with_distinct_t; + DROP TABLE IF EXISTS sum_with_distinct_t; """ sql """ - CREATE TABLE IF NOT EXISTS aggr_with_distinct_t( + CREATE TABLE IF NOT EXISTS sum_with_distinct_t( `id` int(32), `score` int(64) NULL, `name` varchar(64) NULL @@ -38,218 +38,218 @@ suite("push_down_sum_distinct_through_join_one_side") { ); """ - sql "insert into aggr_with_distinct_t values (1, 1, 'a')" - sql "insert into aggr_with_distinct_t values (2, null, 'a')" - sql "insert into aggr_with_distinct_t values (3, 1, null)" - sql "insert into aggr_with_distinct_t values (4, 2, 'b')" - sql "insert into aggr_with_distinct_t values (5, null, 'b')" - sql "insert into aggr_with_distinct_t values (6, 2, null)" - sql "insert into aggr_with_distinct_t values (7, 3, 'c')" - sql "insert into aggr_with_distinct_t values (8, null, 'c')" - sql "insert into aggr_with_distinct_t values (9, 3, null)" - sql "insert into aggr_with_distinct_t values (10, null, null)" - sql "analyze table aggr_with_distinct_t with sync;" + sql "insert into sum_with_distinct_t values (1, 1, 'a')" + sql "insert into sum_with_distinct_t values (2, null, 'a')" + sql "insert into sum_with_distinct_t values (3, 1, null)" + sql "insert into sum_with_distinct_t values (4, 2, 'b')" + sql "insert into sum_with_distinct_t values (5, null, 'b')" + sql "insert into sum_with_distinct_t values (6, 2, null)" + sql "insert into sum_with_distinct_t values (7, 3, 'c')" + sql "insert into sum_with_distinct_t values (8, null, 'c')" + sql "insert into sum_with_distinct_t values (9, 3, null)" + sql "insert into sum_with_distinct_t values (10, null, null)" + sql "analyze table sum_with_distinct_t with sync;" order_qt_groupby_pushdown_basic """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_right_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 right join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_full_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 full join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_semi_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 inner join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_left_anti_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left anti join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_complex_conditions """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_groupby_pushdown_with_aggregate """ - select sum(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score), avg(t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_subquery """ - select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_outer_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_deep_subquery """ - select sum(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from (select * from sum_with_distinct_t) sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_having """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; """ order_qt_groupby_pushdown_mixed_aggregates """ - select sum(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score), sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_multi_table_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id join sum_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_groupby_pushdown_with_order_by """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_groupby_pushdown_multiple_equal_conditions """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_with_aggregate """ - select sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate """ - select t1.name, sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select t1.name, sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select t1.name, sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_groupby_pushdown_with_where_clause """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_groupby_pushdown_varied_aggregates """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_with_order_by_limit """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; """ order_qt_groupby_pushdown_alias_multiple_equal_conditions """ - select sum(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select sum(distinct t1_alias.score) from sum_with_distinct_t t1_alias join sum_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_groupby_pushdown_complex_join_condition """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_groupby_pushdown_function_processed_columns """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_groupby_pushdown_nested_queries """ - select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 20) t1 join (select * from sum_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_basic """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_right_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 right join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 right join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_full_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 full join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 full join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_semi_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 inner join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 inner join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_left_anti_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left anti join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left anti join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_complex_conditions """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_aggregate """ - select sum(distinct t1.score), avg(t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score), avg(t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_subquery """ - select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_outer_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 left join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 left join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_deep_subquery """ - select sum(distinct t1.score) from (select * from (select * from aggr_with_distinct_t) aggr_with_distinct_t where score > 10) t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from (select * from sum_with_distinct_t) sum_with_distinct_t where score > 10) t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_having """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name having sum(distinct t1.score) > 100; """ order_qt_with_hint_groupby_pushdown_mixed_aggregates """ - select sum(distinct t1.score), sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score), sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_multi_table_join """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id join aggr_with_distinct_t t3 on t1.name = t3.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id join sum_with_distinct_t t3 on t1.name = t3.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by t1.name; """ order_qt_with_hint_groupby_pushdown_multiple_equal_conditions """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ - select sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + select sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate """ - select t1.name, sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select t1.name, sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_with_aggregate """ - select t1.name, sum(distinct t1.score), sum(distinct t2.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + select t1.name, sum(distinct t1.score), sum(distinct t2.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_where_clause """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; """ order_qt_with_hint_groupby_pushdown_varied_aggregates """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id group by t1.name; """ order_qt_with_hint_groupby_pushdown_with_order_by_limit """ - select sum(distinct t1.score) from aggr_with_distinct_t t1, aggr_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; + select sum(distinct t1.score) from sum_with_distinct_t t1, sum_with_distinct_t t2 where t1.id = t2.id group by t1.name order by sum(distinct t1.score) limit 10; """ order_qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ - select sum(distinct t1_alias.score) from aggr_with_distinct_t t1_alias join aggr_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + select sum(distinct t1_alias.score) from sum_with_distinct_t t1_alias join sum_with_distinct_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; """ order_qt_with_hint_groupby_pushdown_complex_join_condition """ - select sum(distinct t1.score) from aggr_with_distinct_t t1 join aggr_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + select sum(distinct t1.score) from sum_with_distinct_t t1 join sum_with_distinct_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; """ order_qt_with_hint_groupby_pushdown_nested_queries """ - select sum(distinct t1.score) from (select * from aggr_with_distinct_t where score > 20) t1 join (select * from aggr_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; + select sum(distinct t1.score) from (select * from sum_with_distinct_t where score > 20) t1 join (select * from sum_with_distinct_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ } From eafab7b6c5680bba756e8c35a1b2748df89f59ba Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Fri, 29 Nov 2024 14:49:41 +0800 Subject: [PATCH 08/17] [opt](nereids) support pushdown agg distinct through join --- ...DownAggWithDistinctThroughJoinOneSide.java | 25 ++++++------------- ...distinct_through_join_one_side_cust.groovy | 8 +++--- 2 files changed, 11 insertions(+), 22 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java index 02f6c2b825fbd6..d8db87ddcb6850 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -39,10 +39,8 @@ import com.google.common.collect.Sets; import java.util.ArrayList; -import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; /** @@ -57,7 +55,8 @@ public List buildRules() { .when(agg -> agg.child().child().getOtherJoinConjuncts().isEmpty()) .when(agg -> !agg.isGenerated()) .whenNot(agg -> agg.getAggregateFunctions().isEmpty()) - .whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate)) + .whenNot(agg -> agg.child() + .child(0).children().stream().anyMatch(p -> p instanceof LogicalAggregate)) .when(agg -> { Set funcs = agg.getAggregateFunctions(); if (funcs.size() > 1) { @@ -86,14 +85,18 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate leftFuncs = new ArrayList<>(); List rightFuncs = new ArrayList<>(); + Set leftFuncSlotSet = new HashSet<>(); + Set rightFuncSlotSet = new HashSet<>(); Set newAggOverJoinGroupByKeys = new HashSet<>(); for (AggregateFunction func : agg.getAggregateFunctions()) { Slot slot = (Slot) func.child(0); newAggOverJoinGroupByKeys.add(slot); if (leftJoinOutput.contains(slot)) { leftFuncs.add(func); + leftFuncSlotSet.add(slot); } else if (rightJoinOutput.contains(slot)) { rightFuncs.add(func); + rightFuncSlotSet.add(slot); } else { throw new IllegalStateException("Slot " + slot + " not found in join output"); } @@ -127,27 +130,16 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate leftSlotToOutput = new HashMap<>(); - Map rightSlotToOutput = new HashMap<>(); if (isLeftSideAggDistinct) { leftPushDownGroupBy.add((Slot) leftFuncs.get(0).child(0)); Builder leftAggOutputBuilder = ImmutableList.builder() .addAll(leftPushDownGroupBy); - leftFuncs.forEach(func -> { - Alias alias = func.alias("PDADT_" + func.getName()); - leftSlotToOutput.put((Slot) func.child(0), alias); - }); leftJoin = new LogicalAggregate<>(ImmutableList.copyOf(leftPushDownGroupBy), leftAggOutputBuilder.build(), join.left()); } else { rightPushDownGroupBy.add((Slot) rightFuncs.get(0).child(0)); Builder rightAggOutputBuilder = ImmutableList.builder() .addAll(rightPushDownGroupBy); - rightFuncs.forEach(func -> { - Alias alias = func.alias("PDADT_" + func.getName()); - rightSlotToOutput.put((Slot) func.child(0), alias); - rightAggOutputBuilder.add(alias); - }); rightJoin = new LogicalAggregate<>(ImmutableList.copyOf(rightPushDownGroupBy), rightAggOutputBuilder.build(), join.right()); } @@ -162,10 +154,7 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate Date: Mon, 2 Dec 2024 11:28:22 +0800 Subject: [PATCH 09/17] [opt](nereids) support pushdown agg distinct through join --- .../push_down_count_distinct_through_join_one_side.groovy | 2 +- .../push_down_min_distinct_through_join_one_side.groovy | 2 +- .../push_down_sum_distinct_through_join_one_side.groovy | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index c9e276e50f5001..8ef8e22eef6ab3 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -20,7 +20,7 @@ suite("push_down_count_distinct_through_join_one_side") { sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" sql """ DROP TABLE IF EXISTS count_with_distinct_t; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy index f47a019f0dd33c..615a1acd501cba 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy @@ -20,7 +20,7 @@ suite("push_down_min_distinct_through_join_one_side") { sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" sql """ DROP TABLE IF EXISTS min_with_distinct_t; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy index 708323eec4d731..783f1d8cefb1b5 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy @@ -20,7 +20,7 @@ suite("push_down_sum_distinct_through_join_one_side") { sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" sql """ DROP TABLE IF EXISTS sum_with_distinct_t; From 02ba75d649c2849a9369e6839e05b46cd873c398 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 17:21:07 +0800 Subject: [PATCH 10/17] [opt](nereids) support pushdown agg distinct through join --- ...own_count_distinct_through_join_one_side.out | 17 +++++++++++++++++ ..._count_distinct_through_join_one_side.groovy | 3 +++ 2 files changed, 20 insertions(+) diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out index 4a84ff5117c008..82e1d0c134f722 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out @@ -1,4 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this +-- !shape -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------PhysicalOlapScan[count_with_distinct_t] +------------PhysicalOlapScan[count_with_distinct_t] + +Hint log: +Used: use_PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE +UnUsed: +SyntaxError: + -- !groupby_pushdown_basic -- 1 1 diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index 8ef8e22eef6ab3..011e60efc3eced 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -49,6 +49,9 @@ suite("push_down_count_distinct_through_join_one_side") { sql "insert into count_with_distinct_t values (9, 3, null)" sql "insert into count_with_distinct_t values (10, null, null)" sql "analyze table count_with_distinct_t with sync;" + qt_shape """ + explain shape plan select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; + """ order_qt_groupby_pushdown_basic """ select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ From 805c3126f9ac11308a7ebbf56d71006529a2bc17 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 17:28:55 +0800 Subject: [PATCH 11/17] [opt](nereids) support pushdown agg distinct through join --- .../push_down_count_distinct_through_join_one_side.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index 011e60efc3eced..3cf10786602c9f 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -48,7 +48,7 @@ suite("push_down_count_distinct_through_join_one_side") { sql "insert into count_with_distinct_t values (8, null, 'c')" sql "insert into count_with_distinct_t values (9, 3, null)" sql "insert into count_with_distinct_t values (10, null, null)" - sql "analyze table count_with_distinct_t with sync;" + sql "analyze table count_with_distinct_t with full with sync;" qt_shape """ explain shape plan select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ From 67049f756d05a559c2d07cf1d9e069bdd9a9e7ee Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 17:31:43 +0800 Subject: [PATCH 12/17] [opt](nereids) support pushdown agg distinct through join --- ...ush_down_count_distinct_through_join_one_side.out | 12 ++++++++++++ ..._down_count_distinct_through_join_one_side.groovy | 3 +++ 2 files changed, 15 insertions(+) diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out index 82e1d0c134f722..9c0cc3817fecfb 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out @@ -1,4 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this +-- !select_all -- +1 1 a +10 \N \N +2 \N a +3 1 \N +4 2 b +5 \N b +6 2 \N +7 3 c +8 \N c +9 3 \N + -- !shape -- PhysicalResultSink --hashAgg[GLOBAL] diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index 3cf10786602c9f..7f59753f889261 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -49,6 +49,9 @@ suite("push_down_count_distinct_through_join_one_side") { sql "insert into count_with_distinct_t values (9, 3, null)" sql "insert into count_with_distinct_t values (10, null, null)" sql "analyze table count_with_distinct_t with full with sync;" + order_qt_select_all """ + select * from count_with_distinct_t order by id; + """ qt_shape """ explain shape plan select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ From 2530ff00dd3ad2db743c376d525f87d2f3a2fca8 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Mon, 2 Dec 2024 19:03:33 +0800 Subject: [PATCH 13/17] [opt](nereids) support pushdown agg distinct through join --- .../push_down_aggr_distinct_through_join_one_side_cust.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy index 7b0dcbcf3a35f7..d3e6dd8158e1df 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy @@ -20,7 +20,7 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" + sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION, ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" sql """ DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymd; From 6d4e7058e3141e1a019245aaae808407386a0af1 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Tue, 3 Dec 2024 16:28:09 +0800 Subject: [PATCH 14/17] [opt](nereids) support pushdown agg distinct through join --- ...own_aggr_distinct_through_join_one_side_cust.groovy | 3 +-- ...sh_down_count_distinct_through_join_one_side.groovy | 10 +--------- ...push_down_min_distinct_through_join_one_side.groovy | 3 --- ...push_down_sum_distinct_through_join_one_side.groovy | 3 --- 4 files changed, 2 insertions(+), 17 deletions(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy index d3e6dd8158e1df..94615ce20dedf3 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy @@ -19,8 +19,7 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { sql "SET enable_nereids_planner=true" sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION, ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" + sql "set DISABLE_NEREIDS_RULES='PRUNE_EMPTY_PARTITION, ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" sql """ DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymd; diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy index 7f59753f889261..65a6b15ceccf73 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.groovy @@ -19,9 +19,6 @@ suite("push_down_count_distinct_through_join_one_side") { sql "SET enable_nereids_planner=true" sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" - sql """ DROP TABLE IF EXISTS count_with_distinct_t; """ @@ -49,12 +46,7 @@ suite("push_down_count_distinct_through_join_one_side") { sql "insert into count_with_distinct_t values (9, 3, null)" sql "insert into count_with_distinct_t values (10, null, null)" sql "analyze table count_with_distinct_t with full with sync;" - order_qt_select_all """ - select * from count_with_distinct_t order by id; - """ - qt_shape """ - explain shape plan select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; - """ + order_qt_groupby_pushdown_basic """ select /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/ count(distinct t1.score) from count_with_distinct_t t1, count_with_distinct_t t2 where t1.id = t2.id group by t1.name; """ diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy index 615a1acd501cba..676becbcb6ecfa 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_distinct_through_join_one_side.groovy @@ -19,9 +19,6 @@ suite("push_down_min_distinct_through_join_one_side") { sql "SET enable_nereids_planner=true" sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" - sql """ DROP TABLE IF EXISTS min_with_distinct_t; """ diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy index 783f1d8cefb1b5..799d45ba413fb7 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_distinct_through_join_one_side.groovy @@ -19,9 +19,6 @@ suite("push_down_sum_distinct_through_join_one_side") { sql "SET enable_nereids_planner=true" sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" - sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "set DISABLE_NEREIDS_RULES='ONE_PHASE_AGGREGATE_WITHOUT_DISTINCT, ONE_PHASE_AGGREGATE_SINGLE_DISTINCT_TO_MULTI, PRUNE_EMPTY_PARTITION'" - sql """ DROP TABLE IF EXISTS sum_with_distinct_t; """ From 2c4c431e3934703309c3d3ce8a51a5845b6d7144 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Tue, 3 Dec 2024 16:29:20 +0800 Subject: [PATCH 15/17] [opt](nereids) support pushdown agg distinct through join --- ...n_count_distinct_through_join_one_side.out | 29 ------------------- 1 file changed, 29 deletions(-) diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out index 9c0cc3817fecfb..4a84ff5117c008 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_distinct_through_join_one_side.out @@ -1,33 +1,4 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !select_all -- -1 1 a -10 \N \N -2 \N a -3 1 \N -4 2 b -5 \N b -6 2 \N -7 3 c -8 \N c -9 3 \N - --- !shape -- -PhysicalResultSink ---hashAgg[GLOBAL] -----hashAgg[LOCAL] -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[GLOBAL] ---------------hashAgg[LOCAL] -----------------PhysicalOlapScan[count_with_distinct_t] -------------PhysicalOlapScan[count_with_distinct_t] - -Hint log: -Used: use_PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE -UnUsed: -SyntaxError: - -- !groupby_pushdown_basic -- 1 1 From 5687192dcbc69ed31f90d3f65ae8952ed357cad3 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Tue, 3 Dec 2024 16:41:53 +0800 Subject: [PATCH 16/17] [opt](nereids) support pushdown agg distinct through join --- .../rewrite/PushDownAggWithDistinctThroughJoinOneSide.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java index d8db87ddcb6850..3f9ad609744e21 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownAggWithDistinctThroughJoinOneSide.java @@ -144,7 +144,8 @@ private static LogicalAggregate pushDownAggWithDistinct(LogicalAggregate newAggOverJoin = agg.withChildGroupByAndOutput( ImmutableList.copyOf(newAggOverJoinGroupByKeys), projects, newJoin); From 9e4d53c9d6f3864fb13c4832c832ae71938f4ba9 Mon Sep 17 00:00:00 2001 From: xzj7019 Date: Wed, 4 Dec 2024 15:14:50 +0800 Subject: [PATCH 17/17] [opt](nereids) support pushdown agg distinct through join --- ...distinct_through_join_one_side_cust.groovy | 44 +++++++++---------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy index 94615ce20dedf3..9134d66b76d751 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_aggr_distinct_through_join_one_side_cust.groovy @@ -22,12 +22,12 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { sql "set DISABLE_NEREIDS_RULES='PRUNE_EMPTY_PARTITION, ELIMINATE_GROUP_BY_KEY_BY_UNIFORM'" sql """ - DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymd; - DROP TABLE IF EXISTS dwd_tracking_sensor_init_tmp_ymd; + DROP TABLE IF EXISTS dwd_com_abtest_result_inc_ymds; + DROP TABLE IF EXISTS dwd_tracking_sensor_init_tmp_ymds; """ sql """ - CREATE TABLE `dwd_com_abtest_result_inc_ymd` ( + CREATE TABLE `dwd_com_abtest_result_inc_ymds` ( `app_name` varchar(255) NULL, `user_key` text NULL, `group_name` text NULL, @@ -57,7 +57,7 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { "group_commit_data_bytes" = "134217728" ); - CREATE TABLE `dwd_tracking_sensor_init_tmp_ymd` ( + CREATE TABLE `dwd_tracking_sensor_init_tmp_ymds` ( `ip` varchar(20) NULL, `gz_user_id` text NULL, `dt` date NOT NULL @@ -89,15 +89,15 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { explain { sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + - "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymd.gz_user_id) AS a2c1a830_1," + - "dwd_com_abtest_result_inc_ymd.group_name AS ab1011d6," + - "dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 " + - "FROM dwd_tracking_sensor_init_tmp_ymd " + - "LEFT JOIN dwd_com_abtest_result_inc_ymd " + - "ON dwd_tracking_sensor_init_tmp_ymd.gz_user_id = dwd_com_abtest_result_inc_ymd.user_key " + - "AND dwd_tracking_sensor_init_tmp_ymd.dt = dwd_com_abtest_result_inc_ymd.dt " + - "WHERE dwd_tracking_sensor_init_tmp_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + - "AND dwd_com_abtest_result_inc_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymds.gz_user_id) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymds.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymds.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymds " + + "LEFT JOIN dwd_com_abtest_result_inc_ymds " + + "ON dwd_tracking_sensor_init_tmp_ymds.gz_user_id = dwd_com_abtest_result_inc_ymds.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymds.dt = dwd_com_abtest_result_inc_ymds.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); contains"groupByExpr=[gz_user_id#1, dt#2]" contains"groupByExpr=[gz_user_id#1, dt#2, group_name#5], outputExpr=[gz_user_id#1, dt#2, group_name#5]" @@ -107,15 +107,15 @@ suite("push_down_aggr_distinct_through_join_one_side_cust") { explain { sql("physical PLAN SELECT /*+use_cbo_rule(PUSH_DOWN_AGG_WITH_DISTINCT_THROUGH_JOIN_ONE_SIDE)*/" + - "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymd.ip) AS a2c1a830_1," + - "dwd_com_abtest_result_inc_ymd.group_name AS ab1011d6," + - "dwd_tracking_sensor_init_tmp_ymd.dt AS ad466123 " + - "FROM dwd_tracking_sensor_init_tmp_ymd " + - "LEFT JOIN dwd_com_abtest_result_inc_ymd " + - "ON dwd_tracking_sensor_init_tmp_ymd.gz_user_id = dwd_com_abtest_result_inc_ymd.user_key " + - "AND dwd_tracking_sensor_init_tmp_ymd.dt = dwd_com_abtest_result_inc_ymd.dt " + - "WHERE dwd_tracking_sensor_init_tmp_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + - "AND dwd_com_abtest_result_inc_ymd.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "COUNT(DISTINCT dwd_tracking_sensor_init_tmp_ymds.ip) AS a2c1a830_1," + + "dwd_com_abtest_result_inc_ymds.group_name AS ab1011d6," + + "dwd_tracking_sensor_init_tmp_ymds.dt AS ad466123 " + + "FROM dwd_tracking_sensor_init_tmp_ymds " + + "LEFT JOIN dwd_com_abtest_result_inc_ymds " + + "ON dwd_tracking_sensor_init_tmp_ymds.gz_user_id = dwd_com_abtest_result_inc_ymds.user_key " + + "AND dwd_tracking_sensor_init_tmp_ymds.dt = dwd_com_abtest_result_inc_ymds.dt " + + "WHERE dwd_tracking_sensor_init_tmp_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + + "AND dwd_com_abtest_result_inc_ymds.dt BETWEEN '2024-08-15' AND '2024-08-15' " + "GROUP BY 2, 3 ORDER BY 3 asc limit 10000;"); contains"groupByExpr=[ip#0, gz_user_id#1, dt#2], outputExpr=[ip#0, gz_user_id#1, dt#2]" contains"groupByExpr=[ip#0, dt#2, group_name#5], outputExpr=[ip#0, dt#2, group_name#5]"