diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseCboRuleHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseCboRuleHint.java new file mode 100644 index 00000000000000..34e18ab8568180 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseCboRuleHint.java @@ -0,0 +1,47 @@ +// 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.hint; + +/** + * rule hint. + */ +public class UseCboRuleHint extends Hint { + + private final boolean isNotUseCboRule; + + public UseCboRuleHint(String hintName, boolean isNotUseCboRule) { + super(hintName); + this.isNotUseCboRule = isNotUseCboRule; + } + + public boolean isNotUseCboRule() { + return isNotUseCboRule; + } + + @Override + public String getExplainString() { + StringBuilder out = new StringBuilder(); + if (isNotUseCboRule) { + out.append("no_use_"); + } else { + out.append("use_"); + } + out.append(getHintName()); + return out.toString(); + } +} 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 9d855cc817f5af..e1bd5a0e8c57c8 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 @@ -314,11 +314,11 @@ public class Rewriter extends AbstractBatchJobExecutor { ), topic("Eager aggregation", - topDown( + costBased(topDown( new PushDownAggThroughJoinOneSide(), new PushDownAggThroughJoin() - ), - custom(RuleType.PUSH_DOWN_DISTINCT_THROUGH_JOIN, PushDownDistinctThroughJoin::new) + )), + costBased(custom(RuleType.PUSH_DOWN_DISTINCT_THROUGH_JOIN, PushDownDistinctThroughJoin::new)) ), // this rule should invoke after infer predicate and push down distinct, and before push down limit diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java index 1ff91dfbac9c6e..619eae1b16dbb3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java @@ -20,16 +20,23 @@ import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.cost.Cost; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.UseCboRuleHint; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Optimizer; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.qe.ConnectContext; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Optional; @@ -50,6 +57,12 @@ public CostBasedRewriteJob(List rewriteJobs) { @Override public void execute(JobContext jobContext) { + // checkHint.first means whether it use hint and checkHint.second means what kind of hint it used + Pair checkHint = checkRuleHint(); + // this means it no_use_cbo_rule(xxx) hint + if (checkHint.first && checkHint.second == null) { + return; + } CascadesContext currentCtx = jobContext.getCascadesContext(); CascadesContext skipCboRuleCtx = CascadesContext.newCurrentTreeContext(currentCtx); CascadesContext applyCboRuleCtx = CascadesContext.newCurrentTreeContext(currentCtx); @@ -70,12 +83,73 @@ public void execute(JobContext jobContext) { rewriteJobs, currentCtx.getRewritePlan()); return; } + if (checkHint.first) { + checkHint.second.setStatus(Hint.HintStatus.SUCCESS); + if (((UseCboRuleHint) checkHint.second).isNotUseCboRule()) { + currentCtx.setRewritePlan(applyCboRuleCtx.getRewritePlan()); + } + return; + } // If the candidate applied cbo rule is better, replace the original plan with it. if (appliedCboRuleCost.get().first.getValue() < skipCboRuleCost.get().first.getValue()) { currentCtx.setRewritePlan(applyCboRuleCtx.getRewritePlan()); } } + /** + * check if we have use rule hint or no use rule hint + * return an optional object which checkHint.first means whether it use hint + * and checkHint.second means what kind of hint it used + * example, when we use *+ no_use_cbo_rule(xxx) * the optional would be (true, false) + * which means it use hint and the hint forbid this kind of rule + */ + private Pair checkRuleHint() { + Pair checkResult = Pair.of(false, null); + if (rewriteJobs.get(0) instanceof RootPlanTreeRewriteJob) { + for (Rule rule : ((RootPlanTreeRewriteJob) rewriteJobs.get(0)).getRules()) { + checkResult = checkRuleHintWithHintName(rule.getRuleType()); + if (checkResult.first) { + return checkResult; + } + } + } + if (rewriteJobs.get(0) instanceof CustomRewriteJob) { + checkResult = checkRuleHintWithHintName(((CustomRewriteJob) rewriteJobs.get(0)).getRuleType()); + } + return checkResult; + } + + /** + * for these rules we need use_cbo_rule hint to enable it, otherwise it would be close by default + */ + private static boolean checkBlackList(RuleType ruleType) { + List ruleWhiteList = new ArrayList<>(Arrays.asList( + RuleType.PUSH_DOWN_AGG_THROUGH_JOIN, + RuleType.PUSH_DOWN_AGG_THROUGH_JOIN_ONE_SIDE, + RuleType.PUSH_DOWN_DISTINCT_THROUGH_JOIN)); + if (!ruleWhiteList.isEmpty() && ruleWhiteList.contains(ruleType)) { + return true; + } + return false; + } + + /** + * main mechanism of checkRuleHint + * return an optional object which checkHint.first means whether it use hint + * and checkHint.second means what kind of hint it used + */ + private Pair checkRuleHintWithHintName(RuleType ruleType) { + for (Hint hint : ConnectContext.get().getStatementContext().getHints()) { + if (hint.getHintName().equalsIgnoreCase(ruleType.name())) { + return Pair.of(true, hint); + } + } + if (checkBlackList(ruleType)) { + return Pair.of(true, null); + } + return Pair.of(false, null); + } + @Override public boolean isOnce() { // TODO: currently, we do not support execute it more than once. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java index ae3aa52c4f2094..3e3d33688717db 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CustomRewriteJob.java @@ -79,4 +79,9 @@ public void execute(JobContext context) { public boolean isOnce() { return false; } + + public RuleType getRuleType() { + return ruleType; + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java index d352dfee4a0b20..4949d422613df0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java @@ -120,6 +120,10 @@ public Plan getNewestPlan() { } } + public List getRules() { + return rules; + } + /** use to assemble the rewriting plan */ private static class LinkPlanJob extends Job { LinkPlanJob parentJob; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 077a1a4d881a2d..53dde319d9f9fb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -226,6 +226,7 @@ import org.apache.doris.nereids.properties.SelectHintLeading; import org.apache.doris.nereids.properties.SelectHintOrdered; import org.apache.doris.nereids.properties.SelectHintSetVar; +import org.apache.doris.nereids.properties.SelectHintUseCboRule; import org.apache.doris.nereids.trees.TableSample; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.And; @@ -3126,6 +3127,22 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, SelectHintContext hi case "ordered": hints.put(hintName, new SelectHintOrdered(hintName)); break; + case "use_cbo_rule": + List useRuleParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + useRuleParameters.add(parameterName); + } + hints.put(hintName, new SelectHintUseCboRule(hintName, useRuleParameters, false)); + break; + case "no_use_cbo_rule": + List noUseRuleParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + noUseRuleParameters.add(parameterName); + } + hints.put(hintName, new SelectHintUseCboRule(hintName, noUseRuleParameters, true)); + break; default: break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseCboRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseCboRule.java new file mode 100644 index 00000000000000..47c1ef9dcd7ffb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseCboRule.java @@ -0,0 +1,53 @@ +// 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.properties; + +import java.util.List; + +/** + * select hint CostBasedRule. + */ +public class SelectHintUseCboRule extends SelectHint { + private final List parameters; + + private final boolean isNotUseCboRule; + + public SelectHintUseCboRule(String hintName, List parameters, boolean isNotUseCboRule) { + super(hintName); + this.parameters = parameters; + this.isNotUseCboRule = isNotUseCboRule; + } + + public List getParameters() { + return parameters; + } + + public boolean isNotUseCboRule() { + return isNotUseCboRule; + } + + @Override + public String getHintName() { + return super.getHintName(); + } + + @Override + public String toString() { + return super.getHintName(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java index ac41323f5bc2eb..ea2c9994606502 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/EliminateLogicalSelectHint.java @@ -26,9 +26,11 @@ import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.hint.LeadingHint; import org.apache.doris.nereids.hint.OrderedHint; +import org.apache.doris.nereids.hint.UseCboRuleHint; import org.apache.doris.nereids.properties.SelectHint; import org.apache.doris.nereids.properties.SelectHintLeading; import org.apache.doris.nereids.properties.SelectHintSetVar; +import org.apache.doris.nereids.properties.SelectHintUseCboRule; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; @@ -73,6 +75,8 @@ public Rule build() { } else if (hintName.equalsIgnoreCase("LEADING")) { extractLeading((SelectHintLeading) hint.getValue(), ctx.cascadesContext, ctx.statementContext, selectHintPlan.getHints()); + } else if (hintName.equalsIgnoreCase("USE_CBO_RULE")) { + extractRule((SelectHintUseCboRule) hint.getValue(), ctx.statementContext); } else { logger.warn("Can not process select hint '{}' and skip it", hint.getKey()); } @@ -146,4 +150,12 @@ private void extractLeading(SelectHintLeading selectHint, CascadesContext contex assert (context != null); } + private void extractRule(SelectHintUseCboRule selectHint, StatementContext statementContext) { + // rule hint need added to statementContext only cause it's set in all scopes + for (String parameter : selectHint.getParameters()) { + UseCboRuleHint hint = new UseCboRuleHint(parameter, selectHint.isNotUseCboRule()); + statementContext.addHint(hint); + } + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index f25ac66befc63b..54b6417e708fe8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -1214,7 +1214,7 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { @VariableMgr.VarAttr(name = DISABLE_NEREIDS_RULES, needForward = true) private String disableNereidsRules = ""; - @VariableMgr.VarAttr(name = ENABLE_NEREIDS_RULES, needForward = true) + @VariableMgr.VarAttr(name = ENABLE_NEREIDS_RULES, needForward = true, varType = VariableAnnotation.REMOVED) public String enableNereidsRules = ""; @VariableMgr.VarAttr(name = ENABLE_NEW_COST_MODEL, needForward = true) diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out index 0f52d9c808e0a9..3e3986c75fc91c 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic.out @@ -4,23 +4,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] -----------filter((cast(experiment_id as DOUBLE) = 37.0)) -------------PhysicalOlapScan[shunt_log_com_dd_library] ---------hashAgg[LOCAL] -----------filter((a.event_id = 'ad_click')) -------------PhysicalOlapScan[com_dd_library] +--------filter((a.event_id = 'ad_click')) +----------PhysicalOlapScan[com_dd_library] +--------filter((cast(experiment_id as DOUBLE) = 37.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library] -- !2 -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[com_dd_library] ---------hashAgg[LOCAL] -----------filter((cast(experiment_id as DOUBLE) = 73.0)) -------------PhysicalOlapScan[shunt_log_com_dd_library] +--------PhysicalOlapScan[com_dd_library] +--------filter((cast(experiment_id as DOUBLE) = 73.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library] -- !3 -- PhysicalResultSink @@ -36,8 +32,62 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[shunt_log_com_dd_library] ---------hashAgg[LOCAL] +--------PhysicalOlapScan[shunt_log_com_dd_library] +--------PhysicalOlapScan[com_dd_library] + +-- !with_hint_1 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------filter((a.event_id = 'ad_click')) ----------PhysicalOlapScan[com_dd_library] +--------filter((cast(experiment_id as DOUBLE) = 37.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_2 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------PhysicalOlapScan[com_dd_library] +--------filter((cast(experiment_id as DOUBLE) = 73.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_3 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------PhysicalOlapScan[com_dd_library] +--------filter((cast(experiment_id as DOUBLE) = 73.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_4 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------PhysicalOlapScan[shunt_log_com_dd_library] +--------PhysicalOlapScan[com_dd_library] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join use_push_down_agg_through_join_one_side +SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out index 087b10ea4af8c4..49f1cc9617a090 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/basic_one_side.out @@ -4,9 +4,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] -----------filter((a.event_id = 'ad_click')) -------------PhysicalOlapScan[com_dd_library_one_side] +--------filter((a.event_id = 'ad_click')) +----------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 37.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] @@ -15,8 +14,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[com_dd_library_one_side] +--------PhysicalOlapScan[com_dd_library_one_side] --------filter((cast(experiment_id as DOUBLE) = 73.0)) ----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] @@ -34,7 +32,62 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------PhysicalOlapScan[com_dd_library_one_side] + +-- !with_hint_1 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------filter((a.event_id = 'ad_click')) ----------PhysicalOlapScan[com_dd_library_one_side] +--------filter((cast(experiment_id as DOUBLE) = 37.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_2 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------PhysicalOlapScan[com_dd_library_one_side] +--------filter((cast(experiment_id as DOUBLE) = 73.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_3 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() +--------PhysicalOlapScan[com_dd_library_one_side] +--------filter((cast(experiment_id as DOUBLE) = 73.0)) +----------PhysicalOlapScan[shunt_log_com_dd_library_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_4 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((a.device_id = b.device_id)) otherCondition=() --------PhysicalOlapScan[shunt_log_com_dd_library_one_side] +--------PhysicalOlapScan[com_dd_library_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out index 8c822ef4a6db74..4a12c8d638a4e7 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join.out @@ -4,10 +4,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_left_join -- PhysicalResultSink @@ -38,10 +36,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_left_anti_join -- PhysicalResultSink @@ -72,11 +68,9 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 10)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -91,11 +85,9 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 10)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_having -- PhysicalResultSink @@ -103,10 +95,8 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] -- !groupby_pushdown_mixed_aggregates -- PhysicalResultSink @@ -118,18 +108,13 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -138,20 +123,16 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] -- !groupby_pushdown_multiple_equal_conditions -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_equal_conditions_with_aggregate -- PhysicalResultSink @@ -166,31 +147,25 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((t1.score > 50)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -207,20 +182,16 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] -- !groupby_pushdown_alias_multiple_equal_conditions -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_complex_join_condition -- PhysicalResultSink @@ -243,22 +214,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------filter((count_t.id < 100)) -------------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 20) and (t1.id < 100)) -------------PhysicalOlapScan[count_t] +--------filter((count_t.id < 100)) +----------PhysicalOlapScan[count_t] +--------filter((count_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t] -- !groupby_pushdown_basic -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_left_join -- PhysicalResultSink @@ -289,10 +256,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_left_anti_join -- PhysicalResultSink @@ -315,21 +280,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_subquery -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 10)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -344,11 +305,9 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 10)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_having -- PhysicalResultSink @@ -356,25 +315,18 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -383,41 +335,33 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] -- !groupby_pushdown_multiple_equal_conditions -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[count_t] +--------filter((t1.score > 50)) ----------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[count_t] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -434,10 +378,8 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] -- !groupby_pushdown_complex_join_condition -- PhysicalResultSink @@ -452,10 +394,643 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((count_t.id < 100)) +----------PhysicalOlapScan[count_t] +--------filter((count_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t] + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((count(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((count_t.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[count_t] ---------hashAgg[LOCAL] -----------filter((count_t.score > 20) and (t1.id < 100)) ------------PhysicalOlapScan[count_t] +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((count_t.id < 100)) +----------PhysicalOlapScan[count_t] +--------filter((count_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((count_t.score > 10)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((count(*) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t] +----------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t] +------------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[count_t] +--------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((count_t.id < 100)) +----------PhysicalOlapScan[count_t] +--------filter((count_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out index c3132e181123cf..a2a716e6e6e819 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.out @@ -4,8 +4,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_left_join -- @@ -37,8 +36,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_left_anti_join -- @@ -71,9 +69,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------filter((count_t_one_side.score > 10)) -------------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -89,9 +86,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------filter((count_t_one_side.score > 10)) -------------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_having -- PhysicalResultSink @@ -99,8 +95,7 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] ----------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_mixed_aggregates -- @@ -115,12 +110,9 @@ PhysicalResultSink PhysicalResultSink --hashAgg[LOCAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t_one_side] -------------PhysicalOlapScan[count_t_one_side] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] ------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_with_order_by -- @@ -130,8 +122,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_multiple_equal_conditions -- @@ -139,8 +130,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_equal_conditions_with_aggregate -- @@ -148,18 +138,15 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_equal_conditions_non_aggregate_selection -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -167,10 +154,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink @@ -178,9 +163,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[count_t_one_side] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -197,8 +181,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] ------------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_alias_multiple_equal_conditions -- @@ -206,8 +189,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] --------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_complex_join_condition -- @@ -233,9 +215,8 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((count_t_one_side.id < 100)) ----------PhysicalOlapScan[count_t_one_side] ---------hashAgg[LOCAL] -----------filter((count_t_one_side.score > 20) and (t1.id < 100)) -------------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t_one_side] -- !groupby_pushdown_basic -- PhysicalResultSink @@ -416,3 +397,636 @@ PhysicalResultSink --------filter((count_t_one_side.score > 20) and (t1.id < 100)) ----------PhysicalOlapScan[count_t_one_side] +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((count(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[LOCAL] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] +------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((count_t_one_side.id < 100)) +----------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 10)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((count(*) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[count_t_one_side] +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[LOCAL] +----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] +------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[count_t_one_side] +------------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[count_t_one_side] +--------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((count_t_one_side.id < 100)) +----------PhysicalOlapScan[count_t_one_side] +--------filter((count_t_one_side.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[count_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out index 7725ac6eee304d..281de8ea61b88e 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_max_through_join.out @@ -4,8 +4,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] -- !groupby_pushdown_left_join -- @@ -37,8 +36,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] -- !groupby_pushdown_left_anti_join -- @@ -71,9 +69,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------hashAgg[LOCAL] -----------filter((max_t.score > 10)) -------------PhysicalOlapScan[max_t] +--------filter((max_t.score > 10)) +----------PhysicalOlapScan[max_t] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -89,9 +86,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------hashAgg[LOCAL] -----------filter((max_t.score > 10)) -------------PhysicalOlapScan[max_t] +--------filter((max_t.score > 10)) +----------PhysicalOlapScan[max_t] -- !groupby_pushdown_having -- PhysicalResultSink @@ -99,8 +95,7 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] ----------PhysicalOlapScan[max_t] -- !groupby_pushdown_mixed_aggregates -- @@ -113,15 +108,13 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[max_t] -------------PhysicalOlapScan[max_t] -------PhysicalOlapScan[max_t] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -130,8 +123,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[max_t] +------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] -- !groupby_pushdown_multiple_equal_conditions -- @@ -139,8 +131,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] -- !groupby_pushdown_equal_conditions_with_aggregate -- @@ -148,18 +139,15 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -167,10 +155,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink @@ -178,9 +164,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[max_t] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[max_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[max_t] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -197,8 +182,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[max_t] +------------PhysicalOlapScan[max_t] ------------PhysicalOlapScan[max_t] -- !groupby_pushdown_alias_multiple_equal_conditions -- @@ -206,8 +190,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] --------PhysicalOlapScan[max_t] -- !groupby_pushdown_complex_join_condition -- @@ -233,7 +216,356 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((max_t.id < 100)) ----------PhysicalOlapScan[max_t] +--------filter((max_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[max_t] + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------filter((max_t.score > 10)) +----------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------filter((max_t.score > 10)) +----------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((max(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[max_t] +----------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[max_t] +------------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((max_t.score > 20) and (t1.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[max_t] +------------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[max_t] +--------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((max_t.id < 100)) +----------PhysicalOlapScan[max_t] +--------filter((max_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[max_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out index c0efc8db16df8b..d4efe7df093503 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_min_through_join.out @@ -4,8 +4,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] -- !groupby_pushdown_left_join -- @@ -37,8 +36,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] -- !groupby_pushdown_left_anti_join -- @@ -71,9 +69,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------hashAgg[LOCAL] -----------filter((min_t.score > 10)) -------------PhysicalOlapScan[min_t] +--------filter((min_t.score > 10)) +----------PhysicalOlapScan[min_t] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -89,9 +86,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------hashAgg[LOCAL] -----------filter((min_t.score > 10)) -------------PhysicalOlapScan[min_t] +--------filter((min_t.score > 10)) +----------PhysicalOlapScan[min_t] -- !groupby_pushdown_having -- PhysicalResultSink @@ -99,8 +95,7 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] ----------PhysicalOlapScan[min_t] -- !groupby_pushdown_mixed_aggregates -- @@ -113,15 +108,13 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[min_t] -------------PhysicalOlapScan[min_t] -------PhysicalOlapScan[min_t] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -130,8 +123,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] -- !groupby_pushdown_multiple_equal_conditions -- @@ -139,8 +131,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] -- !groupby_pushdown_equal_conditions_with_aggregate -- @@ -148,18 +139,15 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -167,10 +155,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink @@ -178,9 +164,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[min_t] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[min_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[min_t] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -197,8 +182,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] ------------PhysicalOlapScan[min_t] -- !groupby_pushdown_alias_multiple_equal_conditions -- @@ -206,8 +190,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] --------PhysicalOlapScan[min_t] -- !groupby_pushdown_complex_join_condition -- @@ -233,7 +216,356 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((min_t.id < 100)) ----------PhysicalOlapScan[min_t] +--------filter((min_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[min_t] + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------filter((min_t.score > 10)) +----------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------filter((min_t.score > 10)) +----------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((min(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[min_t] +----------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((min_t.score > 20) and (t1.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[min_t] +------------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[min_t] +--------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((min_t.id < 100)) +----------PhysicalOlapScan[min_t] +--------filter((min_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[min_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out index 2b08c5889dd831..9d20fb8b02dd52 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.out @@ -4,10 +4,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_left_join -- PhysicalResultSink @@ -38,10 +36,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_left_anti_join -- PhysicalResultSink @@ -72,11 +68,9 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 10)) ----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------filter((sum_t.score > 10)) -------------PhysicalOlapScan[sum_t] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -91,11 +85,9 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 10)) ----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------filter((sum_t.score > 10)) -------------PhysicalOlapScan[sum_t] -- !groupby_pushdown_having -- PhysicalResultSink @@ -103,10 +95,8 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t] -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t] +----------PhysicalOlapScan[sum_t] +----------PhysicalOlapScan[sum_t] -- !groupby_pushdown_mixed_aggregates -- PhysicalResultSink @@ -118,18 +108,13 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ----------PhysicalOlapScan[sum_t] -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] +----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -138,20 +123,16 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] +------------PhysicalOlapScan[sum_t] +------------PhysicalOlapScan[sum_t] -- !groupby_pushdown_multiple_equal_conditions -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_equal_conditions_with_aggregate -- PhysicalResultSink @@ -166,41 +147,33 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] +--------PhysicalOlapScan[sum_t] +--------filter((t1.score > 50)) ----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[sum_t] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_with_order_by_limit -- PhysicalResultSink @@ -209,20 +182,16 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t] +------------PhysicalOlapScan[sum_t] +------------PhysicalOlapScan[sum_t] -- !groupby_pushdown_alias_multiple_equal_conditions -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] -- !groupby_pushdown_complex_join_condition -- PhysicalResultSink @@ -245,10 +214,358 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((sum_t.id < 100)) +----------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[sum_t] + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 10)) +----------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 10)) +----------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((sum(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[sum_t] +----------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[sum_t] +----------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((sum_t.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[sum_t] +------------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((sum_t.score > 20) and (t1.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[sum_t] ------------PhysicalOlapScan[sum_t] +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t] +--------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((sum_t.id < 100)) +----------PhysicalOlapScan[sum_t] +--------filter((sum_t.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[sum_t] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join +SyntaxError: + diff --git a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out index f36cb82677d205..5013dc7dbdfb93 100644 --- a/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out +++ b/regression-test/data/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.out @@ -4,8 +4,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_left_join -- @@ -37,8 +36,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_left_anti_join -- @@ -71,9 +69,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[LOCAL] -----------filter((sum_t_one_side.score > 10)) -------------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 10)) +----------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_outer_join -- PhysicalResultSink @@ -89,9 +86,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[LOCAL] -----------filter((sum_t_one_side.score > 10)) -------------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 10)) +----------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_having -- PhysicalResultSink @@ -99,8 +95,7 @@ PhysicalResultSink ----hashAgg[GLOBAL] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------hashAgg[LOCAL] -------------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] ----------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_mixed_aggregates -- @@ -113,15 +108,13 @@ PhysicalResultSink -- !groupby_pushdown_multi_table_join -- PhysicalResultSink ---hashAgg[LOCAL] -----hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] -----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t_one_side] -------------PhysicalOlapScan[sum_t_one_side] -------PhysicalOlapScan[sum_t_one_side] +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_with_order_by -- PhysicalResultSink @@ -130,8 +123,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] ------------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_multiple_equal_conditions -- @@ -139,8 +131,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_equal_conditions_with_aggregate -- @@ -148,18 +139,15 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_equal_conditions_non_aggregate_selection -- PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- @@ -167,10 +155,8 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_with_where_clause -- PhysicalResultSink @@ -178,9 +164,8 @@ PhysicalResultSink ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------PhysicalOlapScan[sum_t_one_side] ---------hashAgg[LOCAL] -----------filter((t1.score > 50)) -------------PhysicalOlapScan[sum_t_one_side] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_varied_aggregates -- PhysicalResultSink @@ -197,8 +182,7 @@ PhysicalResultSink ------hashAgg[GLOBAL] --------hashAgg[LOCAL] ----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -------------hashAgg[LOCAL] ---------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] ------------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_alias_multiple_equal_conditions -- @@ -206,8 +190,7 @@ PhysicalResultSink --hashAgg[GLOBAL] ----hashAgg[LOCAL] ------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() ---------hashAgg[LOCAL] -----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] --------PhysicalOlapScan[sum_t_one_side] -- !groupby_pushdown_complex_join_condition -- @@ -233,7 +216,356 @@ PhysicalResultSink ------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() --------filter((sum_t_one_side.id < 100)) ----------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[sum_t_one_side] + +-- !with_hint_groupby_pushdown_basic -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_right_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_full_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_semi_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_left_anti_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=((t1.name < t2.name)) +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 10)) +----------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_outer_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_deep_subquery -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 10)) +----------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_having -- +PhysicalResultSink +--filter((sum(score) > 100)) +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_mixed_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multi_table_join -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.name = t3.name)) otherCondition=() +--------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------PhysicalOlapScan[sum_t_one_side] +----------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by -- +PhysicalResultSink +--PhysicalQuickSort[MERGE_SORT] +----PhysicalQuickSort[LOCAL_SORT] +------hashAgg[GLOBAL] +--------hashAgg[LOCAL] +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.name = t2.name)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_where_clause -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------filter((t1.score > 50)) +----------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_varied_aggregates -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_with_order_by_limit -- +PhysicalResultSink +--PhysicalTopN[MERGE_SORT] +----PhysicalTopN[LOCAL_SORT] +------hashAgg[GLOBAL] --------hashAgg[LOCAL] -----------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +----------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() ------------PhysicalOlapScan[sum_t_one_side] +------------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_alias_multiple_equal_conditions -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1_alias.id = t2_alias.id) and (t1_alias.name = t2_alias.name)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_complex_join_condition -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id) and (t1.score = t2.score)) otherCondition=(( not (name = name))) +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_function_processed_columns -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[sum_t_one_side] +--------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: + +-- !with_hint_groupby_pushdown_nested_queries -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------filter((sum_t_one_side.id < 100)) +----------PhysicalOlapScan[sum_t_one_side] +--------filter((sum_t_one_side.score > 20) and (t1.id < 100)) +----------PhysicalOlapScan[sum_t_one_side] + +Hint log: +Used: +UnUsed: use_push_down_agg_through_join_one_side +SyntaxError: diff --git a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out index 912ca2554d0abd..1945b9e3d38ba8 100644 --- a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out +++ b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out @@ -332,3 +332,899 @@ PhysicalResultSink ------------------PhysicalProject --------------------PhysicalOlapScan[t2] +-- !with_hint_union_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_complex_conditions -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t1.score > 10)) +----------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t2.name = 'Test')) +----------------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_multi_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_except_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_except_with_filter -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------filter((t1.id > 100)) +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------filter((t2.id < 50)) +----------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_intersect_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_intersect_with_aggregate -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_mixed_set_operators -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalExcept +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalOlapScan[t2] +--------PhysicalDistribute[DistributionSpecHash] +----------PhysicalOlapScan[t3] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t4] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_join_with_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_set_operator_with_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t1.score > 10)) +----------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t2.score < 5)) +----------------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_nested_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t4] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_order_limit -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalTopN[MERGE_SORT] +----------------PhysicalDistribute[DistributionSpecGather] +------------------PhysicalTopN[LOCAL_SORT] +--------------------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_inner_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_left_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_right_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_full_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_left_semi_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_except_with_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------filter((t2.score > 10)) +----------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_intersect_different_types -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_union_complex_aggregate -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecGather] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((t1.id > 100)) +------------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecGather] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((t2.id < 50)) +------------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_union_all_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalUnion +------PhysicalDistribute[DistributionSpecExecutionAny] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecExecutionAny] +--------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_except_complex_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------filter((t2.score > 20)) +------------PhysicalOlapScan[t2] + +Hint log: +Used: use_INFER_SET_OPERATOR_DISTINCT +UnUsed: +SyntaxError: + +-- !with_hint_agg_not_output_groupby -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_complex_conditions -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t1.score > 10)) +----------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t2.name = 'Test')) +----------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_multi_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_except_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_except_with_filter -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------filter((t1.id > 100)) +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------filter((t2.id < 50)) +----------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_intersect_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_intersect_with_aggregate -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecGather] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecGather] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_mixed_set_operators -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalExcept +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------PhysicalOlapScan[t2] +--------PhysicalDistribute[DistributionSpecHash] +----------PhysicalOlapScan[t3] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t4] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_join_with_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_set_operator_with_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t1.score > 10)) +----------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------filter((t2.score < 5)) +----------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_nested_union -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t4] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_order_limit -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalTopN[MERGE_SORT] +----------------PhysicalDistribute[DistributionSpecGather] +------------------PhysicalTopN[LOCAL_SORT] +--------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_inner_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_left_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_right_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_full_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_left_semi_join_combination -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +----------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalOlapScan[t3] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_except_with_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------filter((t2.score > 10)) +----------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_intersect_different_types -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalIntersect +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_complex_aggregate -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecGather] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((t1.id > 100)) +------------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecGather] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((t2.id < 50)) +------------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_union_all_distinct -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalUnion +------PhysicalDistribute[DistributionSpecExecutionAny] +--------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecExecutionAny] +--------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_except_complex_subquery -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----PhysicalExcept +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------PhysicalOlapScan[t1] +------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------filter((t2.score > 20)) +------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + +-- !with_hint_no_agg_not_output_groupby -- +PhysicalResultSink +--PhysicalDistribute[DistributionSpecGather] +----hashAgg[GLOBAL] +------PhysicalDistribute[DistributionSpecHash] +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] + +Hint log: +Used: +UnUsed: use_NO_INFER_SET_OPERATOR_DISTINCT +SyntaxError: + diff --git a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out index 1e6ea0878c77dc..65baed4f1194a0 100644 --- a/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out +++ b/regression-test/data/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.out @@ -6,16 +6,25 @@ PhysicalResultSink ------PhysicalOlapScan[t] ------PhysicalOlapScan[t] +Hint log: +Used: +UnUsed: use_push_down_distinct_through_join +SyntaxError: + -- !basic -- PhysicalResultSink --hashAgg[LOCAL] ----hashJoin[INNER_JOIN] hashCondition=((t1.id = t3.id)) otherCondition=() -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() -----------PhysicalOlapScan[t] -----------PhysicalOlapScan[t] +------hashJoin[INNER_JOIN] hashCondition=((t1.id = t2.id)) otherCondition=() +--------PhysicalOlapScan[t] +--------PhysicalOlapScan[t] ------PhysicalOlapScan[t] +Hint log: +Used: +UnUsed: use_push_down_distinct_through_join +SyntaxError: + -- !basic_sql -- 1 10 diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/basic.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/basic.groovy index 8a721e2ae39280..117d0c01f24838 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/basic.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/basic.groovy @@ -20,9 +20,7 @@ suite("eager_aggregate_basic") { sql "set runtime_filter_mode=OFF" sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql """ @@ -129,4 +127,77 @@ suite("eager_aggregate_basic") { b.experiment_id, a.event_id; """ + + qt_with_hint_1 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join, push_down_agg_through_join_one_side) */ + b.group_id, + COUNT(a.event_id) + from + com_dd_library a + join shunt_log_com_dd_library b on + a.device_id = b.device_id + where + a.event_id = "ad_click" + and b.experiment_id = 37 + group by + b.group_id; + """ + + qt_with_hint_2 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join, push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id) + from + com_dd_library a + join shunt_log_com_dd_library b on + a.device_id = b.device_id + where + b.experiment_id = 73 + group by + b.group_id, + b.experiment_id, + a.event_id; + """ + + qt_with_hint_3 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join, push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id), + date_format(a.time_stamp, '%Y-%m-%d') as dayF + from + com_dd_library a + join shunt_log_com_dd_library b on + a.device_id = b.device_id + where + b.experiment_id = 73 + group by + b.group_id, + b.experiment_id, + a.event_id, + dayF; + """ + + qt_with_hint_4 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join, push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id) + from + com_dd_library a + join shunt_log_com_dd_library b on + a.device_id = b.device_id + group by + b.group_id, + b.experiment_id, + a.event_id; + """ } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/basic_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/basic_one_side.groovy index b93e164d050a4e..f3e6f593aa96c2 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/basic_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/basic_one_side.groovy @@ -21,7 +21,6 @@ suite("eager_aggregate_basic_one_side") { sql "SET enable_fallback_to_original_planner=false" sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql """ @@ -128,4 +127,77 @@ suite("eager_aggregate_basic_one_side") { b.experiment_id, a.event_id; """ + + qt_with_hint_1 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ + b.group_id, + COUNT(a.event_id) + from + com_dd_library_one_side a + join shunt_log_com_dd_library_one_side b on + a.device_id = b.device_id + where + a.event_id = "ad_click" + and b.experiment_id = 37 + group by + b.group_id; + """ + + qt_with_hint_2 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id) + from + com_dd_library_one_side a + join shunt_log_com_dd_library_one_side b on + a.device_id = b.device_id + where + b.experiment_id = 73 + group by + b.group_id, + b.experiment_id, + a.event_id; + """ + + qt_with_hint_3 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id), + date_format(a.time_stamp, '%Y-%m-%d') as dayF + from + com_dd_library_one_side a + join shunt_log_com_dd_library_one_side b on + a.device_id = b.device_id + where + b.experiment_id = 73 + group by + b.group_id, + b.experiment_id, + a.event_id, + dayF; + """ + + qt_with_hint_4 """ + explain shape plan + select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ + a.event_id, + b.experiment_id, + b.group_id, + COUNT(a.event_id) + from + com_dd_library_one_side a + join shunt_log_com_dd_library_one_side b on + a.device_id = b.device_id + group by + b.group_id, + b.experiment_id, + a.event_id; + """ } diff --git a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy index 37cd60009412d2..33b2f888f155f7 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join.groovy @@ -48,8 +48,6 @@ suite("push_down_count_through_join") { sql "insert into count_t values (9, 3, null)" sql "insert into count_t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join" - qt_groupby_pushdown_basic """ explain shape plan select count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name; """ @@ -238,4 +236,193 @@ suite("push_down_count_through_join") { qt_groupby_pushdown_nested_queries """ explain shape plan select count(*) from (select * from count_t where score > 20) t1 join (select * from count_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 left join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 right join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 full join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 inner join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 left anti join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 join count_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score), avg(t1.score) from count_t t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from (select * from count_t where score > 10) t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 left join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from (select * from (select * from count_t) count_t where score > 10) t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name having count(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score), count(*), max(t1.score) from count_t t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 join count_t t2 on t1.id = t2.id join count_t t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ max(t1.score), count(t2.score) from count_t t1 join count_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ t1.name, count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ t1.name, count(t1.score), count(t2.score) from count_t t1, count_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score), avg(t1.id), count(t2.name) from count_t t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name order by count(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1_alias.score) from count_t t1_alias join count_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from count_t t1 join count_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(LENGTH(t1.name)) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(t1.score) from (select * from count_t where score > 20) t1 join (select * from count_t where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + /* COUNT(*) */ + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 left join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 right join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 full join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 inner join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 left anti join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 join count_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from (select * from count_t where score > 10) t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 left join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from (select * from (select * from count_t) count_t where score > 10) t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name having count(*) > 100; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 join count_t t2 on t1.id = t2.id join count_t t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ t1.name, count(*) from count_t t1, count_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*), avg(t1.id), count(t2.name) from count_t t1 join count_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1, count_t t2 where t1.id = t2.id group by t1.name order by count(*) limit 10; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from count_t t1 join count_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ count(*) from (select * from count_t where score > 20) t1 join (select * from count_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_count_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy index aa867cd9118941..595e5fc3706038 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_count_through_join_one_side.groovy @@ -49,8 +49,6 @@ suite("push_down_count_through_join_one_side") { sql "insert into count_t_one_side values (9, 3, null)" sql "insert into count_t_one_side values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" - qt_groupby_pushdown_basic """ explain shape plan select count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name; """ @@ -239,4 +237,193 @@ suite("push_down_count_through_join_one_side") { qt_groupby_pushdown_nested_queries """ explain shape plan select count(*) from (select * from count_t_one_side where score > 20) t1 join (select * from count_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 left join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 right join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 full join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 inner join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 left anti join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score), avg(t1.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from (select * from count_t_one_side where score > 10) t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 left join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from (select * from (select * from count_t_one_side) count_t_one_side where score > 10) t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name having count(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score), count(*), max(t1.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id join count_t_one_side t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score), count(t2.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, count(t1.score), count(t2.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score), avg(t1.id), count(t2.name) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name order by count(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1_alias.score) from count_t_one_side t1_alias join count_t_one_side t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(LENGTH(t1.name)) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(t1.score) from (select * from count_t_one_side where score > 20) t1 join (select * from count_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; + """ + + /* COUNT(*) */ + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 left join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 right join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 full join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 inner join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 left anti join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from (select * from count_t_one_side where score > 10) t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 left join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from (select * from (select * from count_t_one_side) count_t_one_side where score > 10) t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name having count(*) > 100; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id join count_t_one_side t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*), avg(t1.id), count(t2.name) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1, count_t_one_side t2 where t1.id = t2.id group by t1.name order by count(*) limit 10; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from count_t_one_side t1 join count_t_one_side t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ count(*) from (select * from count_t_one_side where score > 20) t1 join (select * from count_t_one_side 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_max_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy index 26772637fe7b7b..1b3d2d44fcd087 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_max_through_join.groovy @@ -48,8 +48,6 @@ suite("push_down_max_through_join") { sql "insert into max_t values (9, 3, null)" sql "insert into max_t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" - qt_groupby_pushdown_basic """ explain shape plan select max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name; """ @@ -153,4 +151,109 @@ suite("push_down_max_through_join") { qt_groupby_pushdown_nested_queries """ explain shape plan select max(t1.score) from (select * from max_t where score > 20) t1 join (select * from max_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 left join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 right join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 full join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 inner join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 left anti join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 join max_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score), avg(t1.score) from max_t t1 join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from (select * from max_t where score > 10) t1 join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 left join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from (select * from (select * from max_t) max_t where score > 10) t1 join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name having max(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score), count(*), sum(t1.score) from max_t t1 join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 join max_t t2 on t1.id = t2.id join max_t t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score), max(t2.score) from max_t t1 join max_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, max(t1.score), max(t2.score) from max_t t1, max_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score), avg(t1.id), count(t2.name) from max_t t1 join max_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name order by max(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1_alias.score) from max_t t1_alias join max_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from max_t t1 join max_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(LENGTH(t1.name)) from max_t t1, max_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score) from (select * from max_t where score > 20) t1 join (select * from max_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_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy index 7942fbd28c4cb7..c93dfe53d80c1e 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_min_through_join.groovy @@ -48,8 +48,6 @@ suite("push_down_min_through_join") { sql "insert into min_t values (9, 3, null)" sql "insert into min_t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" - qt_groupby_pushdown_basic """ explain shape plan select min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name; """ @@ -153,4 +151,108 @@ suite("push_down_min_through_join") { qt_groupby_pushdown_nested_queries """ explain shape plan select min(t1.score) from (select * from min_t where score > 20) t1 join (select * from min_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 left join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 right join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 full join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 inner join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 left anti join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 join min_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score), avg(t1.score) from min_t t1 join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from (select * from min_t where score > 10) t1 join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 left join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from (select * from (select * from min_t) min_t where score > 10) t1 join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name having min(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score), count(*), sum(t1.score) from min_t t1 join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 join min_t t2 on t1.id = t2.id join min_t t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score), min(t2.score) from min_t t1 join min_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, min(t1.score), min(t2.score) from min_t t1, min_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score), avg(t1.id), count(t2.name) from min_t t1 join min_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name order by min(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1_alias.score) from min_t t1_alias join min_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from min_t t1 join min_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(LENGTH(t1.name)) from min_t t1, min_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ min(t1.score) from (select * from min_t where score > 20) t1 join (select * from min_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_through_join.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy index 95736d264759dd..4f9470860b25ee 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join.groovy @@ -48,8 +48,6 @@ suite("push_down_sum_through_join") { sql "insert into sum_t values (9, 3, null)" sql "insert into sum_t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join" - qt_groupby_pushdown_basic """ explain shape plan select sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name; """ @@ -153,4 +151,108 @@ suite("push_down_sum_through_join") { qt_groupby_pushdown_nested_queries """ explain shape plan select sum(t1.score) from (select * from sum_t where score > 20) t1 join (select * from sum_t where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 left join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 right join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 full join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 inner join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 left anti join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 join sum_t t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score), avg(t1.score) from sum_t t1 join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from (select * from sum_t where score > 10) t1 join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 left join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from (select * from (select * from sum_t) sum_t where score > 10) t1 join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name having sum(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score), count(*), max(t1.score) from sum_t t1 join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 join sum_t t2 on t1.id = t2.id join sum_t t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ max(t1.score), sum(t2.score) from sum_t t1 join sum_t t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ t1.name, sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ t1.name, sum(t1.score), sum(t2.score) from sum_t t1, sum_t t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score), count(t2.name) from sum_t t1 join sum_t t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name order by sum(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1_alias.score) from sum_t t1_alias join sum_t t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from sum_t t1 join sum_t t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(LENGTH(t1.name)) from sum_t t1, sum_t t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join) */ sum(t1.score) from (select * from sum_t where score > 20) t1 join (select * from sum_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_through_join_one_side.groovy b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy index fecf141026114d..1910c294de42b9 100644 --- a/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy +++ b/regression-test/suites/nereids_rules_p0/eager_aggregate/push_down_sum_through_join_one_side.groovy @@ -47,9 +47,7 @@ suite("push_down_sum_through_join_one_side") { sql "insert into sum_t_one_side values (8, null, 'c')" sql "insert into sum_t_one_side values (9, 3, null)" sql "insert into sum_t_one_side values (10, null, null)" - - sql "SET ENABLE_NEREIDS_RULES=push_down_agg_through_join_one_side" - + qt_groupby_pushdown_basic """ explain shape plan select sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name; """ @@ -153,4 +151,108 @@ suite("push_down_sum_through_join_one_side") { qt_groupby_pushdown_nested_queries """ explain shape plan select sum(t1.score) from (select * from sum_t_one_side where score > 20) t1 join (select * from sum_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; """ -} \ No newline at end of file + + qt_with_hint_groupby_pushdown_basic """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 left join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_right_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 right join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_full_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 full join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_semi_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 inner join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_left_anti_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 left anti join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id and t1.name < t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score), avg(t1.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from (select * from sum_t_one_side where score > 10) t1 join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_outer_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 left join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_deep_subquery """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from (select * from (select * from sum_t_one_side) sum_t_one_side where score > 10) t1 join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_having """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name having sum(t1.score) > 100; + """ + + qt_with_hint_groupby_pushdown_mixed_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score), count(*), max(t1.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_multi_table_join """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id join sum_t_one_side t3 on t1.name = t3.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name order by t1.name; + """ + + qt_with_hint_groupby_pushdown_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ max(t1.score), sum(t2.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_equal_conditions_non_aggregate_selection_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ t1.name, sum(t1.score), sum(t2.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id and t1.name = t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_where_clause """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id and t1.score > 50 group by t1.name; + """ + + qt_with_hint_groupby_pushdown_varied_aggregates """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score), avg(t1.id), count(t2.name) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_with_order_by_limit """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name order by sum(t1.score) limit 10; + """ + + qt_with_hint_groupby_pushdown_alias_multiple_equal_conditions """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1_alias.score) from sum_t_one_side t1_alias join sum_t_one_side t2_alias on t1_alias.id = t2_alias.id and t1_alias.name = t2_alias.name group by t1_alias.name; + """ + + qt_with_hint_groupby_pushdown_complex_join_condition """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from sum_t_one_side t1 join sum_t_one_side t2 on t1.id = t2.id and t1.score = t2.score and t1.name <> t2.name group by t1.name; + """ + + qt_with_hint_groupby_pushdown_function_processed_columns """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(LENGTH(t1.name)) from sum_t_one_side t1, sum_t_one_side t2 where t1.id = t2.id group by t1.name; + """ + + qt_with_hint_groupby_pushdown_nested_queries """ + explain shape plan select /*+ USE_CBO_RULE(push_down_agg_through_join_one_side) */ sum(t1.score) from (select * from sum_t_one_side where score > 20) t1 join (select * from sum_t_one_side where id < 100) t2 on t1.id = t2.id group by t1.name; + """ +} diff --git a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy index 930c6578a81030..c93485aabf2cea 100644 --- a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy +++ b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy @@ -172,4 +172,189 @@ suite("test_infer_set_operator_distinct") { qt_agg_not_output_groupby """ explain shape plan select sum(t1.score) from t1 group by t1.id union select sum(t2.score) from t2 group by t2.id """ + + qt_with_hint_union_distinct """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2; + """ + + qt_with_hint_union_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 where t1.score > 10 union select * from t2 where t2.name = 'Test'; + """ + + qt_with_hint_multi_union """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 union select * from t3; + """ + + qt_with_hint_except_distinct """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 except select * from t2; + """ + + qt_with_hint_except_with_filter """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 where t1.id > 100 except select * from t2 where t2.id < 50; + """ + + qt_with_hint_intersect_distinct """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 intersect select * from t2; + """ + + qt_with_hint_intersect_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ AVG(t1.score) from t1 intersect select SUM(t2.score) from t2; + """ + + qt_with_hint_mixed_set_operators """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 except select * from t3 intersect select * from t4; + """ + + qt_with_hint_join_with_union """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_set_operator_with_subquery """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from (select * from t1 where score > 10) sub1 union select * from (select * from t2 where score < 5) sub2; + """ + + qt_with_hint_nested_union """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from ((select * from t1 union select * from t2) union (select * from t3 union select * from t4)) sub1; + """ + + qt_with_hint_union_order_limit """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 order by id limit 10; + """ + + qt_with_hint_union_inner_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_union_left_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 left join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_union_right_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 right join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_union_full_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 full join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_union_left_semi_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 left semi join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_except_with_subquery """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 except select * from (select * from t2 where score > 10) sub1; + """ + + qt_with_hint_intersect_different_types """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ name from t1 intersect select name from t2; + """ + + qt_with_hint_union_complex_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ MAX(score) from t1 where id > 100 union select MIN(score) from t2 where id < 50; + """ + + qt_with_hint_union_all_distinct """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from t1 union all select * from t2; + """ + + qt_with_hint_except_complex_subquery """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ * from (select id, name from t1) sub1 except select * from (select id, name from t2 where score > 20) sub2; + """ + + qt_with_hint_agg_not_output_groupby """ + explain shape plan select /*+ USE_CBO_RULE(INFER_SET_OPERATOR_DISTINCT) */ sum(t1.score) from t1 group by t1.id union select sum(t2.score) from t2 group by t2.id + """ + + qt_with_hint_no_union_distinct """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2; + """ + + qt_with_hint_no_union_complex_conditions """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 where t1.score > 10 union select * from t2 where t2.name = 'Test'; + """ + + qt_with_hint_no_multi_union """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 union select * from t3; + """ + + qt_with_hint_no_except_distinct """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 except select * from t2; + """ + + qt_with_hint_no_except_with_filter """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 where t1.id > 100 except select * from t2 where t2.id < 50; + """ + + qt_with_hint_no_intersect_distinct """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 intersect select * from t2; + """ + + qt_with_hint_no_intersect_with_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ AVG(t1.score) from t1 intersect select SUM(t2.score) from t2; + """ + + qt_with_hint_no_mixed_set_operators """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 except select * from t3 intersect select * from t4; + """ + + qt_with_hint_no_join_with_union """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_set_operator_with_subquery """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from (select * from t1 where score > 10) sub1 union select * from (select * from t2 where score < 5) sub2; + """ + + qt_with_hint_no_nested_union """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from ((select * from t1 union select * from t2) union (select * from t3 union select * from t4)) sub1; + """ + + qt_with_hint_no_union_order_limit """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 union select * from t2 order by id limit 10; + """ + + qt_with_hint_no_union_inner_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_union_left_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 left join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_union_right_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 right join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_union_full_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 full join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_union_left_semi_join_combination """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ t1.* from t1 left semi join t2 on t1.id = t2.id union select * from t3; + """ + + qt_with_hint_no_except_with_subquery """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 except select * from (select * from t2 where score > 10) sub1; + """ + + qt_with_hint_no_intersect_different_types """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ name from t1 intersect select name from t2; + """ + + qt_with_hint_no_union_complex_aggregate """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ MAX(score) from t1 where id > 100 union select MIN(score) from t2 where id < 50; + """ + + qt_with_hint_no_union_all_distinct """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from t1 union all select * from t2; + """ + + qt_with_hint_no_except_complex_subquery """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ * from (select id, name from t1) sub1 except select * from (select id, name from t2 where score > 20) sub2; + """ + + qt_with_hint_no_agg_not_output_groupby """ + explain shape plan select /*+ USE_CBO_RULE(NO_INFER_SET_OPERATOR_DISTINCT) */ sum(t1.score) from t1 group by t1.id union select sum(t2.score) from t2 group by t2.id + """ + } diff --git a/regression-test/suites/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.groovy b/regression-test/suites/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.groovy index 1903f99e7a2f93..ef572d518b1389 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_distinct_through_join/push_down_distinct_through_join.groovy @@ -49,22 +49,19 @@ suite("push_down_distinct_through_join") { sql "insert into t values (9, 3, null)" sql "insert into t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_distinct_through_join" - qt_basic_not """ - explain shape plan select distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id) t; + explain shape plan select /*+ USE_CBO_RULE(push_down_distinct_through_join) */ distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id) t; """ qt_basic """ - explain shape plan select distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; + explain shape plan select /*+ USE_CBO_RULE(push_down_distinct_through_join) */ distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; """ order_qt_basic_sql """ - select distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; + select /*+ USE_CBO_RULE(push_down_distinct_through_join) */ distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; """ - sql "SET ENABLE_NEREIDS_RULES=''" order_qt_basic_sql_disable """ - select distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; + select /*+ USE_CBO_RULE(no_push_down_distinct_through_join) */ distinct * from (select t1.id from t t1 join t t2 on t1.id = t2.id join t t3 on t1.id = t3.id) t; """ -} \ No newline at end of file +} diff --git a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy index f333cbee8962b8..779a2762e9bdb1 100644 --- a/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy +++ b/regression-test/suites/nereids_rules_p0/push_down_limit_distinct/push_down_limit_distinct.groovy @@ -49,9 +49,7 @@ suite("push_down_limit_distinct_through_join") { sql "insert into t values (9, 3, null)" sql "insert into t values (10, null, null)" - sql "SET ENABLE_NEREIDS_RULES=push_down_limit_distinct_through_join" - qt_basic """ explain shape plan (select id from t t1 union select id from t t2) limit 0, 200; """ -} \ No newline at end of file +} diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query1.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query1.groovy index 4178ed67a17bb2..7b017f03eb754a 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query1.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query1.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=true' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_1 ''' explain shape plan with customer_total_return as diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.groovy index e33c81351c782e..2644305aa88f70 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=true' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_30 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.groovy index 081e42e9d7c04b..2940d4484c54ac 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=true' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_81 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query1.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query1.groovy index 29e437b73d3505..bf8fdd1e6a8647 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query1.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query1.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=false' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_1 ''' explain shape plan with customer_total_return as diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.groovy index c8933a02e28825..a23548c99fa51e 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=false' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_30 ''' explain shape plan diff --git a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.groovy b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.groovy index 164924f40a1da4..4880b8e24f8bed 100644 --- a/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.groovy +++ b/regression-test/suites/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.groovy @@ -32,7 +32,7 @@ sql 'set enable_runtime_filter_prune=false' sql 'set broadcast_row_count_limit = 30000000' sql 'set enable_nereids_timeout = false' sql 'SET enable_pipeline_engine = true' - sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" qt_ds_shape_81 ''' explain shape plan