diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index f30fcaadf3fa2f..61701bfbf078a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -51,6 +51,8 @@ import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVVersionSnapshot; +import org.apache.doris.nereids.hint.Hint; +import org.apache.doris.nereids.hint.UseMvHint; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; import org.apache.doris.resource.Tag; @@ -513,6 +515,99 @@ public Map getVisibleIndexIdToMeta() { return visibleMVs; } + public Long getBestMvIdWithHint(List orderedMvs) { + Optional useMvHint = getUseMvHint("USE_MV"); + Optional noUseMvHint = getUseMvHint("NO_USE_MV"); + if (useMvHint.isPresent() && noUseMvHint.isPresent()) { + if (noUseMvHint.get().getNoUseMVName(this.name).contains(useMvHint.get().getUseMvName(this.name))) { + String errorMsg = "conflict mv exist in use_mv and no_use_mv in the same time" + + useMvHint.get().getUseMvName(this.name); + useMvHint.get().setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.get().setErrorMessage(errorMsg); + noUseMvHint.get().setStatus(Hint.HintStatus.SYNTAX_ERROR); + noUseMvHint.get().setErrorMessage(errorMsg); + } + return getMvIdWithUseMvHint(useMvHint.get(), orderedMvs); + } else if (useMvHint.isPresent()) { + return getMvIdWithUseMvHint(useMvHint.get(), orderedMvs); + } else if (noUseMvHint.isPresent()) { + return getMvIdWithNoUseMvHint(noUseMvHint.get(), orderedMvs); + } + return orderedMvs.get(0); + } + + private Long getMvIdWithUseMvHint(UseMvHint useMvHint, List orderedMvs) { + if (useMvHint.isAllMv()) { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("use_mv hint should only have one mv in one table: " + + this.name); + return orderedMvs.get(0); + } else { + String mvName = useMvHint.getUseMvName(this.name); + if (mvName != null) { + if (mvName.equals("`*`")) { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("use_mv hint should only have one mv in one table: " + + this.name); + return orderedMvs.get(0); + } + Long choosedIndexId = indexNameToId.get(mvName); + if (orderedMvs.contains(choosedIndexId)) { + useMvHint.setStatus(Hint.HintStatus.SUCCESS); + return choosedIndexId; + } else { + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("do not have mv: " + mvName + " in table: " + this.name); + } + } + } + return orderedMvs.get(0); + } + + private Long getMvIdWithNoUseMvHint(UseMvHint noUseMvHint, List orderedMvs) { + if (noUseMvHint.isAllMv()) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + return getBaseIndex().getId(); + } else { + List mvNames = noUseMvHint.getNoUseMVName(this.name); + Set forbiddenIndexIds = Sets.newHashSet(); + for (int i = 0; i < mvNames.size(); i++) { + if (mvNames.get(i).equals("`*`")) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + return getBaseIndex().getId(); + } + if (hasMaterializedIndex(mvNames.get(i))) { + Long forbiddenIndexId = indexNameToId.get(mvNames.get(i)); + forbiddenIndexIds.add(forbiddenIndexId); + } else { + noUseMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + noUseMvHint.setErrorMessage("do not have mv: " + mvNames.get(i) + " in table: " + this.name); + break; + } + } + for (int i = 0; i < orderedMvs.size(); i++) { + if (forbiddenIndexIds.contains(orderedMvs.get(i))) { + noUseMvHint.setStatus(Hint.HintStatus.SUCCESS); + } else { + return orderedMvs.get(i); + } + } + } + return orderedMvs.get(0); + } + + private Optional getUseMvHint(String useMvName) { + for (Hint hint : ConnectContext.get().getStatementContext().getHints()) { + if (hint.isSyntaxError()) { + continue; + } + if (hint.getHintName().equalsIgnoreCase(useMvName)) { + return Optional.of((UseMvHint) hint); + } + } + return Optional.empty(); + } + public List getVisibleIndex() { Optional partition = idToPartition.values().stream().findFirst(); if (!partition.isPresent()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 0c3516f41c3002..3adc9e43f2abf2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -148,7 +148,7 @@ public class CascadesContext implements ScheduleContext { */ private CascadesContext(Optional parent, Optional currentTree, StatementContext statementContext, Plan plan, Memo memo, - CTEContext cteContext, PhysicalProperties requireProperties) { + CTEContext cteContext, PhysicalProperties requireProperties, boolean isLeadingDisableJoinReorder) { this.parent = Objects.requireNonNull(parent, "parent should not null"); this.currentTree = Objects.requireNonNull(currentTree, "currentTree should not null"); this.statementContext = Objects.requireNonNull(statementContext, "statementContext should not null"); @@ -169,6 +169,10 @@ private CascadesContext(Optional parent, Optional curren } else { this.isEnableExprTrace = false; } + if (parent.isPresent()) { + this.tables = parent.get().tables; + } + this.isLeadingDisableJoinReorder = isLeadingDisableJoinReorder; } /** @@ -177,7 +181,7 @@ private CascadesContext(Optional parent, Optional curren public static CascadesContext initContext(StatementContext statementContext, Plan initPlan, PhysicalProperties requireProperties) { return newContext(Optional.empty(), Optional.empty(), statementContext, - initPlan, new CTEContext(), requireProperties); + initPlan, new CTEContext(), requireProperties, false); } /** @@ -186,14 +190,15 @@ public static CascadesContext initContext(StatementContext statementContext, public static CascadesContext newContextWithCteContext(CascadesContext cascadesContext, Plan initPlan, CTEContext cteContext) { return newContext(Optional.of(cascadesContext), Optional.empty(), - cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY + cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY, + cascadesContext.isLeadingDisableJoinReorder ); } public static CascadesContext newCurrentTreeContext(CascadesContext context) { return CascadesContext.newContext(context.getParent(), context.getCurrentTree(), context.getStatementContext(), context.getRewritePlan(), context.getCteContext(), - context.getCurrentJobContext().getRequiredProperties()); + context.getCurrentJobContext().getRequiredProperties(), context.isLeadingDisableJoinReorder); } /** @@ -202,14 +207,14 @@ public static CascadesContext newCurrentTreeContext(CascadesContext context) { public static CascadesContext newSubtreeContext(Optional subtree, CascadesContext context, Plan plan, PhysicalProperties requireProperties) { return CascadesContext.newContext(Optional.of(context), subtree, context.getStatementContext(), - plan, context.getCteContext(), requireProperties); + plan, context.getCteContext(), requireProperties, context.isLeadingDisableJoinReorder); } private static CascadesContext newContext(Optional parent, Optional subtree, StatementContext statementContext, Plan initPlan, CTEContext cteContext, - PhysicalProperties requireProperties) { + PhysicalProperties requireProperties, boolean isLeadingDisableJoinReorder) { return new CascadesContext(parent, subtree, statementContext, initPlan, null, - cteContext, requireProperties); + cteContext, requireProperties, isLeadingDisableJoinReorder); } public CascadesContext getRoot() { 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/hint/UseMvHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseMvHint.java new file mode 100644 index 00000000000000..5e37bdc27603bc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/hint/UseMvHint.java @@ -0,0 +1,144 @@ +// 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; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * rule hint. + */ +public class UseMvHint extends Hint { + + private final boolean isUseMv; + + private final boolean isAllMv; + + private final List parameters; + + private final Map useMvTableColumnMap; + + private final Map> noUseMvTableColumnMap; + + /** + * constructor of use mv hint + * @param hintName use mv + * @param parameters original parameters + * @param isUseMv use_mv hint or no_use_mv hint + * @param isAllMv should all mv be controlled + */ + public UseMvHint(String hintName, List parameters, boolean isUseMv, boolean isAllMv) { + super(hintName); + this.isUseMv = isUseMv; + this.isAllMv = isAllMv; + this.parameters = parameters; + this.useMvTableColumnMap = initUseMvTableColumnMap(parameters); + this.noUseMvTableColumnMap = initNoUseMvTableColumnMap(parameters); + } + + private Map initUseMvTableColumnMap(List parameters) { + Map tempUseMvTableColumnMap = new HashMap<>(); + if (!isUseMv) { + return tempUseMvTableColumnMap; + } + if (parameters.size() % 2 == 1) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("parameter of use_mv hint must be in pairs"); + return tempUseMvTableColumnMap; + } + for (int i = 0; i < parameters.size(); i += 2) { + String tableName = parameters.get(i); + String columnName = parameters.get(i + 1); + if (tempUseMvTableColumnMap.containsKey(tableName)) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("use_mv hint should only have one mv in one table: " + + tableName + "." + columnName); + break; + } + tempUseMvTableColumnMap.put(tableName, columnName); + } + return tempUseMvTableColumnMap; + } + + private Map> initNoUseMvTableColumnMap(List parameters) { + Map> tempNoUseMvTableColumnMap = new HashMap<>(); + if (isUseMv) { + return tempNoUseMvTableColumnMap; + } + if (parameters.size() % 2 == 1) { + this.setStatus(HintStatus.SYNTAX_ERROR); + this.setErrorMessage("parameter of no_use_mv hint must be in pairs"); + return tempNoUseMvTableColumnMap; + } + for (int i = 0; i < parameters.size(); i += 2) { + String tableName = parameters.get(i); + String columnName = parameters.get(i + 1); + if (tempNoUseMvTableColumnMap.containsKey(tableName)) { + tempNoUseMvTableColumnMap.get(tableName).add(columnName); + } else { + List list = new ArrayList<>(); + list.add(columnName); + tempNoUseMvTableColumnMap.put(tableName, list); + } + } + return tempNoUseMvTableColumnMap; + } + + public boolean isUseMv() { + return isUseMv; + } + + public boolean isAllMv() { + return isAllMv; + } + + public String getUseMvName(String tableName) { + return useMvTableColumnMap.get(tableName); + } + + public List getNoUseMVName(String tableName) { + return noUseMvTableColumnMap.get(tableName); + } + + @Override + public String getExplainString() { + StringBuilder out = new StringBuilder(); + if (isUseMv) { + out.append("use_mv"); + } else { + out.append("no_use_mv"); + } + if (!parameters.isEmpty()) { + out.append("("); + for (int i = 0; i < parameters.size(); i++) { + if (i % 2 == 0) { + out.append(parameters.get(i)); + } else { + out.append("."); + out.append(parameters.get(i)); + out.append(" "); + } + } + out.append(")"); + } + + 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 413a410ac59932..a8860bf6cb377c 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 @@ -328,11 +328,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..dc13412031a7d4 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 859e942b380b7f..551efa1026d695 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 @@ -129,6 +129,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 61521bd7681692..0d635a3a1b332c 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 @@ -224,6 +224,8 @@ 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.properties.SelectHintUseMv; import org.apache.doris.nereids.trees.TableSample; import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.And; @@ -3094,7 +3096,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List hints = Maps.newLinkedHashMap(); + ImmutableList.Builder hints = ImmutableList.builder(); for (ParserRuleContext hintContext : hintContexts) { SelectHintContext selectHintContext = (SelectHintContext) hintContext; for (HintStatementContext hintStatement : selectHintContext.hintStatements) { @@ -3116,7 +3118,7 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List leadingParameters = new ArrayList(); @@ -3126,17 +3128,53 @@ private LogicalPlan withSelectHint(LogicalPlan logicalPlan, List useRuleParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + useRuleParameters.add(parameterName); + } + hints.add(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.add(new SelectHintUseCboRule(hintName, noUseRuleParameters, true)); + break; + case "use_mv": + List useIndexParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + if (kv.key != null) { + useIndexParameters.add(parameterName); + } + } + hints.add(new SelectHintUseMv(hintName, useIndexParameters, true)); + break; + case "no_use_mv": + List noUseIndexParameters = new ArrayList(); + for (HintAssignmentContext kv : hintStatement.parameters) { + String parameterName = visitIdentifierOrText(kv.key); + if (kv.key != null) { + noUseIndexParameters.add(parameterName); + } + } + hints.add(new SelectHintUseMv(hintName, noUseIndexParameters, false)); break; default: break; } } } - return new LogicalSelectHint<>(hints, logicalPlan); + return new LogicalSelectHint<>(hints.build(), logicalPlan); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java index 8e8889f5e62df2..31a205d5ed5054 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java @@ -59,7 +59,7 @@ public Plan visitUnboundResultSink(UnboundResultSink unboundResu public Plan visitLogicalSubQueryAlias(LogicalSubQueryAlias alias, StatementContext context) { if (alias.child() instanceof LogicalSelectHint - && ((LogicalSelectHint) alias.child()).isIncludeLeading()) { + && ((LogicalSelectHint) alias.child()).isIncludeHint("Leading")) { aliasQueries.add((LogicalSubQueryAlias) alias); List tableName = new ArrayList<>(); tableName.add(alias.getAlias()); 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/properties/SelectHintUseMv.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseMv.java new file mode 100644 index 00000000000000..35ce25fb4f47c6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/SelectHintUseMv.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 UseMv. + */ +public class SelectHintUseMv extends SelectHint { + private final List parameters; + + private final boolean isUseMv; + + public SelectHintUseMv(String hintName, List parameters, boolean isUseMv) { + super(hintName); + this.parameters = parameters; + this.isUseMv = isUseMv; + } + + public List getParameters() { + return parameters; + } + + public boolean isUseMv() { + return isUseMv; + } + + @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/AnalyzeCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java index 0fe083c1e93230..129b0860a74ee4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java @@ -72,6 +72,7 @@ public Rule build() { CascadesContext outerCascadesCtx = CascadesContext.newContextWithCteContext( ctx.cascadesContext, logicalCTE.child(), result.first); outerCascadesCtx.newAnalyzer().analyze(); + ctx.cascadesContext.setLeadingDisableJoinReorder(outerCascadesCtx.isLeadingDisableJoinReorder()); Plan root = outerCascadesCtx.getRewritePlan(); // should construct anchor from back to front, because the cte behind depends on the front for (int i = result.second.size() - 1; i >= 0; i--) { 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..13eee57637bfc4 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,13 @@ 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.hint.UseMvHint; 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.properties.SelectHintUseMv; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; @@ -41,7 +45,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -55,10 +58,10 @@ public class EliminateLogicalSelectHint extends OneRewriteRuleFactory { public Rule build() { return logicalSelectHint().thenApply(ctx -> { LogicalSelectHint selectHintPlan = ctx.root; - for (Entry hint : selectHintPlan.getHints().entrySet()) { - String hintName = hint.getKey(); + for (SelectHint hint : selectHintPlan.getHints()) { + String hintName = hint.getHintName(); if (hintName.equalsIgnoreCase("SET_VAR")) { - setVar((SelectHintSetVar) hint.getValue(), ctx.statementContext); + setVar((SelectHintSetVar) hint, ctx.statementContext); } else if (hintName.equalsIgnoreCase("ORDERED")) { try { ctx.cascadesContext.getConnectContext().getSessionVariable() @@ -71,10 +74,16 @@ public Rule build() { ctx.cascadesContext.getHintMap().put("Ordered", ordered); ctx.statementContext.addHint(ordered); } else if (hintName.equalsIgnoreCase("LEADING")) { - extractLeading((SelectHintLeading) hint.getValue(), ctx.cascadesContext, - ctx.statementContext, selectHintPlan.getHints()); + extractLeading((SelectHintLeading) hint, ctx.cascadesContext, + ctx.statementContext, selectHintPlan); + } else if (hintName.equalsIgnoreCase("USE_CBO_RULE")) { + extractRule((SelectHintUseCboRule) hint, ctx.statementContext); + } else if (hintName.equalsIgnoreCase("USE_MV")) { + extractMv((SelectHintUseMv) hint, ConnectContext.get().getStatementContext()); + } else if (hintName.equalsIgnoreCase("NO_USE_MV")) { + extractMv((SelectHintUseMv) hint, ConnectContext.get().getStatementContext()); } else { - logger.warn("Can not process select hint '{}' and skip it", hint.getKey()); + logger.warn("Can not process select hint '{}' and skip it", hint.getHintName()); } } return selectHintPlan.child(); @@ -112,7 +121,7 @@ private void setVar(SelectHintSetVar selectHint, StatementContext context) { } private void extractLeading(SelectHintLeading selectHint, CascadesContext context, - StatementContext statementContext, Map hints) { + StatementContext statementContext, LogicalSelectHint selectHintPlan) { LeadingHint hint = new LeadingHint("Leading", selectHint.getParameters(), selectHint.toString()); if (context.getHintMap().get("Leading") != null) { hint.setStatus(Hint.HintStatus.SYNTAX_ERROR); @@ -135,7 +144,8 @@ private void extractLeading(SelectHintLeading selectHint, CascadesContext contex if (!hint.isSyntaxError()) { hint.setStatus(Hint.HintStatus.SUCCESS); } - if (hints.get("ordered") != null || ConnectContext.get().getSessionVariable().isDisableJoinReorder() + if (selectHintPlan.isIncludeHint("Ordered") + || ConnectContext.get().getSessionVariable().isDisableJoinReorder() || context.isLeadingDisableJoinReorder()) { context.setLeadingJoin(false); hint.setStatus(Hint.HintStatus.UNUSED); @@ -146,4 +156,27 @@ 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); + } + } + + private void extractMv(SelectHintUseMv selectHint, StatementContext statementContext) { + boolean isAllMv = selectHint.getParameters().isEmpty(); + UseMvHint useMvHint = new UseMvHint(selectHint.getHintName(), selectHint.getParameters(), + selectHint.isUseMv(), isAllMv); + for (Hint hint : statementContext.getHints()) { + if (hint.getHintName().equals(selectHint.getHintName())) { + hint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + hint.setErrorMessage("only one " + selectHint.getHintName() + " hint is allowed"); + useMvHint.setStatus(Hint.HintStatus.SYNTAX_ERROR); + useMvHint.setErrorMessage("only one " + selectHint.getHintName() + " hint is allowed"); + } + } + statementContext.addHint(useMvHint); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java index f1b2d1233cff2a..13c0600689fde1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/mv/AbstractSelectMaterializedIndexRule.java @@ -266,7 +266,7 @@ protected static long selectBestIndex( .thenComparing(rid -> (Long) rid)) .collect(Collectors.toList()); - return sortedIndexIds.get(0); + return table.getBestMvIdWithHint(sortedIndexIds); } protected static List matchPrefixMost( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java index 127889ea7ed471..a33e2194131c8d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSelectHint.java @@ -29,10 +29,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; @@ -44,9 +42,9 @@ public class LogicalSelectHint extends LogicalUnary implements BlockFuncDepsPropagation { - private final Map hints; + private final ImmutableList hints; - public LogicalSelectHint(Map hints, CHILD_TYPE child) { + public LogicalSelectHint(ImmutableList hints, CHILD_TYPE child) { this(hints, Optional.empty(), Optional.empty(), child); } @@ -57,19 +55,29 @@ public LogicalSelectHint(Map hints, CHILD_TYPE child) { * @param logicalProperties logicalProperties is use for compute output * @param child child plan */ - public LogicalSelectHint(Map hints, + public LogicalSelectHint(ImmutableList hints, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_SELECT_HINT, groupExpression, logicalProperties, child); - this.hints = ImmutableMap.copyOf(Objects.requireNonNull(hints, "hints can not be null")); + this.hints = ImmutableList.copyOf(Objects.requireNonNull(hints, "hints can not be null")); } - public Map getHints() { + public List getHints() { return hints; } - public boolean isIncludeLeading() { - return hints.containsKey("leading"); + /** + * check if current select hint include some hint + * @param hintName hint name + * @return boolean which indicate have hint + */ + public boolean isIncludeHint(String hintName) { + for (SelectHint hint : hints) { + if (hint.getHintName().equalsIgnoreCase(hintName)) { + return true; + } + } + return false; } @Override @@ -107,9 +115,9 @@ public List computeOutput() { @Override public String toString() { - String hintStr = this.hints.entrySet() + String hintStr = this.hints .stream() - .map(entry -> entry.getValue().toString()) + .map(hint -> hint.toString()) .collect(Collectors.joining(", ")); return "LogicalSelectHint (" + hintStr + ")"; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java index 0b2ed8069ade83..f96fe7e918c410 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/joinorder/joinhint/DistributeHintTest.java @@ -28,13 +28,12 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; -import com.google.common.collect.Maps; +import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Set; public class DistributeHintTest extends TPCHTestBase { @@ -85,15 +84,15 @@ public void testHintJoin() { } private Plan generateLeadingHintPlan(int tableNum, Plan childPlan) { - Map hints = Maps.newLinkedHashMap(); + ImmutableList.Builder hints = ImmutableList.builder(); List leadingParameters = new ArrayList(); for (int i = 0; i < tableNum; i++) { leadingParameters.add(String.valueOf(i)); } Collections.shuffle(leadingParameters); System.out.println("LeadingHint: " + leadingParameters.toString()); - hints.put("leading", new SelectHintLeading("leading", leadingParameters)); - return new LogicalSelectHint<>(hints, childPlan); + hints.add(new SelectHintLeading("Leading", leadingParameters)); + return new LogicalSelectHint<>(hints.build(), childPlan); } private void randomTest(int tableNum, int edgeNum, boolean withJoinHint, boolean withLeading) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java index abe82c858d4e38..a91c0dd47126fc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java @@ -68,6 +68,11 @@ public class AnalyzeCTETest extends TestWithFeService implements MemoPatternMatc + "cte2 AS (SELECT sk FROM cte1 WHERE sk < 3)" + "SELECT * FROM cte1 JOIN cte2 ON cte1.sk = cte2.sk"; + private final String cteLeadingJoin = "WITH cte1 AS (SELECT /*+ leading(supplier customer) */ s_suppkey AS sk " + + "FROM supplier join customer on c_nation = s_nation), " + + "cte2 AS (SELECT sk FROM cte1 WHERE sk < 3)" + + "SELECT /*+ leading(cte2 cte1) */ * FROM cte1 JOIN cte2 ON cte1.sk = cte2.sk"; + private final String cteReferToAnotherOne = "WITH V1 AS (SELECT s_suppkey FROM supplier), " + "V2 AS (SELECT s_suppkey FROM V1)" + "SELECT * FROM V2"; @@ -128,6 +133,15 @@ public List getExplorationRules() { } } + @Test + public void testLeadingCte() throws Exception { + StatementScopeIdGenerator.clear(); + StatementContext statementContext = MemoTestUtils.createStatementContext(connectContext, cteLeadingJoin); + NereidsPlanner planner = new NereidsPlanner(statementContext); + planner.planWithLock(parser.parseSingle(cteLeadingJoin), PhysicalProperties.ANY); + Assertions.assertTrue(planner.getCascadesContext().isLeadingDisableJoinReorder()); + } + @Test public void testCTEInHavingAndSubquery() { diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out index 14332a7762c448..be41df2b567ea8 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out @@ -31,13 +31,14 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 ------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] ------------------------PhysicalProject ---------------------------filter((customer_address.ca_state = 'AR')) -----------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_state = 'AR')) +------------------------------PhysicalOlapScan[customer_address] --------------PhysicalDistribute[DistributionSpecReplicated] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query64.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query64.out index ae4207e726381c..aa57e97704fcd4 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query64.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query64.out @@ -4,48 +4,48 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --PhysicalCteProducer ( cteId=CTEId#1 ) ----hashAgg[GLOBAL] ------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF20 p_promo_sk->[ss_promo_sk] -----------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF19 ca_address_sk->[c_current_addr_sk] +--------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF20 cs_item_sk->[i_item_sk,sr_item_sk,ss_item_sk] +----------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF19 p_promo_sk->[ss_promo_sk] ------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF18 s_store_sk->[ss_store_sk] ---------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF15 sr_item_sk->[ss_item_sk];RF16 sr_item_sk->[cr_item_sk,cs_item_sk,i_item_sk];RF17 sr_ticket_number->[ss_ticket_number] -----------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF14 d_date_sk->[c_first_shipto_date_sk] -------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF13 ca_address_sk->[ss_addr_sk] ---------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF12 hd_demo_sk->[c_current_hdemo_sk] -----------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[c_first_sales_date_sk] -------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF10 hd_demo_sk->[ss_hdemo_sk] ---------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF9 i_item_sk->[cr_item_sk,cs_item_sk,ss_item_sk] -----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF7 cd_demo_sk->[c_current_cdemo_sk] ---------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF6 cs_item_sk->[ss_item_sk] -----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[ss_cdemo_sk] -------------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF5 RF6 RF8 RF9 RF10 RF13 RF15 RF17 RF18 RF20 ---------------------------------------PhysicalOlapScan[customer] apply RFs: RF7 RF11 RF12 RF14 RF19 -------------------------------------PhysicalOlapScan[customer_demographics] -----------------------------------filter((sale > (2 * refund))) -------------------------------------hashAgg[GLOBAL] ---------------------------------------hashAgg[LOCAL] -----------------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cr_item_sk->[cs_item_sk];RF3 cr_order_number->[cs_order_number] -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF9 RF16 -------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF9 RF16 ---------------------------------PhysicalOlapScan[customer_demographics] -------------------------------filter(d_year IN (1999, 2000)) +--------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF17 d_date_sk->[ss_sold_date_sk] +----------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF16 ib_income_band_sk->[hd_income_band_sk] +------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF15 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[c_current_hdemo_sk] +----------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF13 ca_address_sk->[ss_addr_sk] +------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF10 sr_item_sk->[ss_item_sk];RF11 sr_item_sk->[i_item_sk];RF12 sr_ticket_number->[ss_ticket_number] +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF9 hd_demo_sk->[ss_hdemo_sk] +----------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF8 i_item_sk->[ss_item_sk] +------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[c_first_shipto_date_sk] +--------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF6 cd_demo_sk->[ss_cdemo_sk] +----------------------------------hashJoin[INNER_JOIN] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 RF8 RF9 RF10 RF12 RF13 RF17 RF18 RF19 RF20 +------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------------hashJoin[INNER_JOIN] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[c_first_sales_date_sk] +----------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF4 RF7 RF14 RF15 +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------PhysicalOlapScan[customer_demographics] --------------------------------PhysicalOlapScan[date_dim] -----------------------------filter((item.i_current_price <= 58.00) and (item.i_current_price >= 49.00) and i_color IN ('blush', 'lace', 'lawn', 'misty', 'orange', 'pink')) -------------------------------PhysicalOlapScan[item] apply RFs: RF16 ---------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF1 ib_income_band_sk->[hd_income_band_sk] -----------------------------PhysicalOlapScan[household_demographics] apply RFs: RF1 -----------------------------PhysicalOlapScan[income_band] -------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[INNER_JOIN] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF0 ib_income_band_sk->[hd_income_band_sk] -------------------------PhysicalOlapScan[household_demographics] apply RFs: RF0 -------------------------PhysicalOlapScan[income_band] +------------------------------filter((item.i_current_price <= 58.00) and (item.i_current_price >= 49.00) and i_color IN ('blush', 'lace', 'lawn', 'misty', 'orange', 'pink')) +--------------------------------PhysicalOlapScan[item] apply RFs: RF11 RF20 +----------------------------hashJoin[INNER_JOIN] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF2 ib_income_band_sk->[hd_income_band_sk] +------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF2 +------------------------------PhysicalOlapScan[income_band] +--------------------------PhysicalOlapScan[store_returns] apply RFs: RF20 +------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalOlapScan[household_demographics] apply RFs: RF16 --------------------PhysicalOlapScan[customer_address] +------------------PhysicalOlapScan[income_band] +----------------filter(d_year IN (1999, 2000)) ------------------PhysicalOlapScan[date_dim] -----------------PhysicalOlapScan[store_returns] --------------PhysicalOlapScan[store] -------------PhysicalOlapScan[customer_address] -----------PhysicalOlapScan[promotion] +------------PhysicalOlapScan[promotion] +----------filter((sale > (2 * refund))) +------------hashAgg[GLOBAL] +--------------hashAgg[LOCAL] +----------------hashJoin[INNER_JOIN] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_item_sk->[cs_item_sk];RF1 cr_order_number->[cs_order_number] +------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +------------------PhysicalOlapScan[catalog_returns] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalQuickSort[LOCAL_SORT] @@ -56,7 +56,7 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) ------------PhysicalCteConsumer ( cteId=CTEId#1 ) Hint log: -Used: leading(catalog_sales catalog_returns ) leading({ store_sales { { customer d2 } cd2 } } cd1 d3 item { hd1 ib1 } store_returns ad1 hd2 ad2 ib2 d1 store promotion cs_ui ) leading(cs1 cs2 ) -UnUsed: +Used: leading(catalog_sales shuffle catalog_returns ) leading({ store_sales { { customer d2 } cd2 } } cd1 d3 item { hd1 ib1 } store_returns ad1 hd2 ad2 ib2 d1 store promotion cs_ui ) leading(cs1 shuffle cs2 ) +UnUsed: SyntaxError: diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out index 62e2e1dc4026a8..85c1c1fc6ec637 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out @@ -27,19 +27,19 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------PhysicalProject ------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] --------------PhysicalProject -----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +----------------hashJoin[INNER_JOIN] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ctr_customer_sk->[c_customer_sk] ------------------PhysicalDistribute[DistributionSpecHash] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF2 +----------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 ------------------------PhysicalDistribute[DistributionSpecHash] --------------------------PhysicalProject ----------------------------filter((customer_address.ca_state = 'TN')) ------------------------------PhysicalOlapScan[customer_address] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 --------------PhysicalDistribute[DistributionSpecReplicated] ----------------hashAgg[GLOBAL] ------------------PhysicalDistribute[DistributionSpecHash] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query83.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query83.out index b87ee9dbc205d5..7b639f77eb2ef2 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query83.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query83.out @@ -5,90 +5,91 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] --------PhysicalProject -----------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = cr_items.item_id)) otherCondition=() build RFs:RF13 item_id->[i_item_id] +----------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF13 item_id->[i_item_id,i_item_id] ------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[cr_item_sk] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[cr_returned_date_sk] +--------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = cr_items.item_id)) otherCondition=() build RFs:RF12 item_id->[i_item_id] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF11 d_date_sk->[sr_returned_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF11 RF12 +------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF10 i_item_sk->[sr_item_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF10 RF11 +--------------------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] apply RFs: RF12 RF13 ----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF10 d_date->[d_date] +--------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF9 d_date->[d_date] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF10 +------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF9 ----------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF9 d_week_seq->[d_week_seq] +--------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF8 d_week_seq->[d_week_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF9 +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF8 ----------------------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------------------PhysicalProject --------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) ----------------------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] apply RFs: RF13 -------------PhysicalProject ---------------hashJoin[INNER_JOIN] hashCondition=((sr_items.item_id = wr_items.item_id)) otherCondition=() build RFs:RF8 item_id->[i_item_id] ----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute[DistributionSpecHash] ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF7 i_item_sk->[sr_item_sk] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashJoin[INNER_JOIN] hashCondition=((store_returns.sr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[sr_returned_date_sk] +--------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cr_returned_date_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN] hashCondition=((catalog_returns.cr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[cr_item_sk] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF6 RF7 +----------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF6 RF7 --------------------------------PhysicalDistribute[DistributionSpecReplicated] ----------------------------------PhysicalProject -------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 ---------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------PhysicalOlapScan[item] apply RFs: RF13 +----------------------------PhysicalDistribute[DistributionSpecReplicated] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF8 -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[wr_item_sk] -----------------------------PhysicalDistribute[DistributionSpecHash] -------------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[wr_returned_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 RF3 ---------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF5 d_date->[d_date] ----------------------------------PhysicalProject -------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF5 +----------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------PhysicalProject +--------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF4 d_week_seq->[d_week_seq] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 ---------------------------------------------PhysicalDistribute[DistributionSpecReplicated] -----------------------------------------------PhysicalProject -------------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) ---------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------------------PhysicalProject +--------------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) +----------------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[wr_returned_date_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((web_returns.wr_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[wr_item_sk] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] apply RFs: RF2 RF3 ----------------------------PhysicalDistribute[DistributionSpecHash] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_date = date_dim.d_date)) otherCondition=() build RFs:RF1 d_date->[d_date] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------PhysicalProject +----------------------------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((date_dim.d_week_seq = date_dim.d_week_seq)) otherCondition=() build RFs:RF0 d_week_seq->[d_week_seq] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF0 +------------------------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------------------------PhysicalProject +----------------------------------------filter(d_date IN ('2001-07-13', '2001-09-10', '2001-11-16')) +------------------------------------------PhysicalOlapScan[date_dim] Hint log: Used: leading(sr_items cr_items wr_items ) diff --git a/regression-test/data/nereids_p0/hint/multi_leading.out b/regression-test/data/nereids_p0/hint/multi_leading.out index 8e12a465bf2663..2659fc32911149 100644 --- a/regression-test/data/nereids_p0/hint/multi_leading.out +++ b/regression-test/data/nereids_p0/hint/multi_leading.out @@ -1,20 +1,4 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !sql1_1 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((cte.c1 = t1.c1)) otherCondition=() -----------hashJoin[INNER_JOIN] hashCondition=((cte.c1 = t2.c2)) otherCondition=() -------------filter((t2.c2 > 300)) ---------------PhysicalOlapScan[t2] -------------PhysicalDistribute[DistributionSpecHash] ---------------filter((cte.c1 > 300)) -----------------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecHash] -------------filter((t1.c1 > 300)) ---------------PhysicalOlapScan[t1] - -- !sql1_2 -- PhysicalResultSink --hashAgg[GLOBAL] @@ -88,19 +72,6 @@ SyntaxError: -- !sql1_res_4 -- 1686 --- !sql2_1 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((alias1.c1 = t3.c3)) otherCondition=() -----------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecHash] ---------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalOlapScan[t3] - -- !sql2_2 -- PhysicalResultSink --hashAgg[GLOBAL] @@ -167,25 +138,6 @@ SyntaxError: -- !sql2_res_4 -- 3103 --- !sql3_1 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((alias1.c1 = cte.c11)) otherCondition=() -----------hashJoin[INNER_JOIN] hashCondition=((alias1.c1 = t3.c3)) otherCondition=() -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalOlapScan[t2] -------------PhysicalDistribute[DistributionSpecHash] ---------------PhysicalOlapScan[t3] -----------PhysicalDistribute[DistributionSpecHash] -------------hashJoin[INNER_JOIN] hashCondition=((cte.c1 = t2.c2)) otherCondition=() ---------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalOlapScan[t2] - -- !sql3_2 -- PhysicalResultSink --hashAgg[GLOBAL] @@ -270,22 +222,6 @@ SyntaxError: -- !sql3_res_4 -- 10411 --- !sql4_0 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((alias1.c1 = t3.c3)) otherCondition=() -----------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashJoin[INNER_JOIN] hashCondition=((t2.c2 = t4.c4)) otherCondition=() -----------------PhysicalOlapScan[t2] -----------------PhysicalDistribute[DistributionSpecHash] -------------------PhysicalOlapScan[t4] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalOlapScan[t3] - -- !sql4_1 -- PhysicalResultSink --hashAgg[GLOBAL] @@ -394,51 +330,10 @@ SyntaxError: -- !sql4_res_7 -- 6224 --- !sql5_1 -- -PhysicalCteAnchor ( cteId=CTEId#0 ) ---PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalOlapScan[t1] ---PhysicalResultSink -----PhysicalDistribute[DistributionSpecGather] -------NestedLoopJoin[INNER_JOIN](cast(sum(c11) as DOUBLE) > (cast(0.05 as DOUBLE) * avg(c11))) ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------PhysicalDistribute[DistributionSpecReplicated] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = cte.c11)) otherCondition=() -------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) - -- !sql5_2 -- -PhysicalCteAnchor ( cteId=CTEId#0 ) ---PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalOlapScan[t1] ---PhysicalResultSink -----PhysicalDistribute[DistributionSpecGather] -------NestedLoopJoin[INNER_JOIN](cast(sum(c11) as DOUBLE) > (cast(0.05 as DOUBLE) * avg(c11))) ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------PhysicalDistribute[DistributionSpecReplicated] -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecGather] ---------------hashAgg[LOCAL] -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = cte.c11)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------PhysicalOlapScan[t1] - -Hint log: -Used: leading(cte t1 ) -UnUsed: -SyntaxError: +0 +1 +3 +4 +1 diff --git a/regression-test/data/nereids_p0/hint/test_distribute.out b/regression-test/data/nereids_p0/hint/test_distribute.out index 5379183bebe581..9a167d97d28893 100644 --- a/regression-test/data/nereids_p0/hint/test_distribute.out +++ b/regression-test/data/nereids_p0/hint/test_distribute.out @@ -1594,17 +1594,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1616,17 +1617,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1638,17 +1640,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1660,17 +1663,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1682,17 +1686,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1704,17 +1709,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1726,17 +1732,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -1748,17 +1755,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 [shuffle]_3 @@ -1787,17 +1795,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1809,17 +1818,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1831,17 +1841,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -1853,17 +1864,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1875,17 +1887,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1897,17 +1910,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -1919,17 +1933,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -1941,17 +1956,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 [shuffle]_3 @@ -2393,17 +2409,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2415,17 +2432,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2437,17 +2455,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2459,17 +2478,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2481,17 +2501,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2503,17 +2524,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2525,17 +2547,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -2547,17 +2570,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 [shuffle]_3 @@ -2569,34 +2593,36 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] -- !select25_2 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2608,17 +2634,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2630,17 +2657,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2652,17 +2680,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2674,18 +2703,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2697,18 +2727,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2720,17 +2751,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -2742,18 +2774,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 [shuffle]_3 @@ -2782,18 +2815,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: @@ -2805,17 +2839,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -2827,18 +2862,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: @@ -2850,18 +2886,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2873,17 +2910,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2895,18 +2933,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -2918,18 +2957,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -2941,18 +2981,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_3 @@ -2981,18 +3022,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: @@ -3004,17 +3046,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -3026,18 +3069,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: @@ -3049,18 +3093,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3072,17 +3117,18 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalProject -----------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3094,18 +3140,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3117,18 +3164,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -3140,18 +3188,19 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalProject -------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() ------------PhysicalProject ---------------PhysicalOlapScan[t3] +--------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_3 @@ -3163,32 +3212,34 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t3] -- !select28_2 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -3200,16 +3251,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -3221,16 +3273,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 @@ -3242,16 +3295,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3263,16 +3317,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3284,16 +3339,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +--------------PhysicalProject +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecHash] --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 @@ -3305,16 +3361,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecHash] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecReplicated] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [shuffle]_2 [broadcast]_3 @@ -3326,16 +3383,17 @@ PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] ---------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() -------------PhysicalProject ---------------PhysicalOlapScan[t1] -------------PhysicalDistribute[DistributionSpecReplicated] +--------PhysicalProject +----------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() --------------PhysicalProject -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalProject ---------------PhysicalOlapScan[t3] +----------------PhysicalOlapScan[t1] +--------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------PhysicalOlapScan[t2] +------------PhysicalDistribute[DistributionSpecHash] +--------------PhysicalProject +----------------PhysicalOlapScan[t3] Hint log: Used: [broadcast]_2 [shuffle]_3 @@ -5842,16 +5900,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select41_2 -- PhysicalResultSink @@ -5862,26 +5922,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -5897,26 +5960,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -5942,16 +6008,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -5967,26 +6035,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6002,26 +6073,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6037,26 +6111,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6072,26 +6149,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6107,26 +6187,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6152,16 +6235,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6177,26 +6262,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6212,26 +6300,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6247,26 +6338,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6282,26 +6376,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select42_2 -- PhysicalResultSink @@ -6312,26 +6409,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6347,26 +6447,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6382,26 +6485,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6417,26 +6523,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6452,26 +6561,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6487,26 +6599,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6522,26 +6637,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6558,26 +6676,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6593,26 +6714,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6629,26 +6753,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6664,26 +6791,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6700,26 +6830,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -6746,16 +6879,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select43_2 -- PhysicalResultSink @@ -6777,16 +6912,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6803,26 +6940,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -6849,16 +6989,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6875,26 +7017,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6911,26 +7056,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -6947,26 +7095,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -6993,16 +7144,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7019,26 +7172,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7065,16 +7221,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7091,26 +7249,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7137,16 +7298,18 @@ PhysicalResultSink ------------------------PhysicalProject --------------------------PhysicalOlapScan[t2] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7163,26 +7326,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7208,16 +7374,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select44_2 -- PhysicalResultSink @@ -7228,26 +7396,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7263,26 +7434,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7308,16 +7482,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7333,26 +7509,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7368,26 +7547,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7403,26 +7585,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7438,26 +7623,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7473,26 +7661,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7518,16 +7709,18 @@ PhysicalResultSink ----------------------PhysicalProject ------------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7543,26 +7736,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7578,26 +7774,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7613,26 +7812,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7648,26 +7850,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select45_2 -- PhysicalResultSink @@ -7678,26 +7883,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7713,26 +7921,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7748,26 +7959,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7783,26 +7997,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7818,26 +8035,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7853,26 +8073,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecReplicated] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecReplicated] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -7888,26 +8111,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7924,26 +8150,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7959,26 +8188,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -7995,26 +8227,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8030,26 +8265,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8066,26 +8304,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] +------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8111,16 +8352,18 @@ PhysicalResultSink ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select46_2 -- PhysicalResultSink @@ -8131,27 +8374,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8165,28 +8411,31 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8212,16 +8461,18 @@ PhysicalResultSink ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8237,27 +8488,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8271,28 +8525,31 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8308,27 +8565,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8344,27 +8604,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8378,28 +8641,31 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8425,16 +8691,18 @@ PhysicalResultSink ------------------PhysicalProject --------------------PhysicalOlapScan[t1] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8450,27 +8718,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8486,27 +8757,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8522,27 +8796,30 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((alias2.c2 = t3.c3)) otherCondition=() -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] +----------------PhysicalProject +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8558,26 +8835,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t1] +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select47_2 -- PhysicalResultSink @@ -8586,29 +8866,32 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8624,26 +8907,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t1] +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8659,26 +8945,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t1] +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8692,29 +8981,32 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8730,26 +9022,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t1] +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -8763,29 +9058,32 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: @@ -8799,29 +9097,32 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject -------------------PhysicalOlapScan[t3] +------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8838,26 +9139,29 @@ PhysicalResultSink ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalDistribute[DistributionSpecHash] -------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[t1] +------------------------PhysicalOlapScan[t2] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8873,26 +9177,29 @@ PhysicalResultSink ----------NestedLoopJoin[CROSS_JOIN] ------------PhysicalProject --------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() -----------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalProject ---------------------PhysicalOlapScan[t2] -------------------PhysicalDistribute[DistributionSpecHash] +----------------PhysicalProject +------------------hashJoin[LEFT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalProject -----------------------PhysicalOlapScan[t1] +----------------------PhysicalOlapScan[t2] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------PhysicalProject +------------------------PhysicalOlapScan[t1] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject --------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -8906,81 +9213,48 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------PhysicalOlapScan[t1] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------PhysicalProject -------------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] -------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] - -Hint log: -Used: [shuffle]_2 -UnUsed: -SyntaxError: - --- !select47_12 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------PhysicalProject -----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t1] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------PhysicalProject -----------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 UnUsed: SyntaxError: --- !select47_13 -- +-- !select47_12 -- PhysicalResultSink --hashAgg[GLOBAL] ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ----------------PhysicalProject ------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() --------------------PhysicalDistribute[DistributionSpecHash] @@ -8989,20 +9263,62 @@ PhysicalResultSink --------------------PhysicalDistribute[DistributionSpecHash] ----------------------PhysicalProject ------------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] +--------------------PhysicalProject +----------------------PhysicalLimit[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] + +Hint log: +Used: [shuffle]_2 +UnUsed: +SyntaxError: + +-- !select47_13 -- +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute[DistributionSpecGather] +------hashAgg[LOCAL] +--------PhysicalProject +----------NestedLoopJoin[CROSS_JOIN] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t1] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9016,27 +9332,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] -- !select48_2 -- PhysicalResultSink @@ -9045,27 +9364,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecReplicated] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecHash] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9079,27 +9401,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9113,27 +9438,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9147,27 +9475,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9181,27 +9512,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] -----------------PhysicalDistribute[DistributionSpecHash] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] +----------------PhysicalDistribute[DistributionSpecReplicated] +------------------PhysicalProject +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9215,27 +9549,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecReplicated] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecReplicated] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecReplicated] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecReplicated] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [broadcast]_2 @@ -9249,27 +9586,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9283,27 +9623,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9317,27 +9660,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9351,27 +9697,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9385,27 +9734,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 @@ -9419,27 +9771,30 @@ PhysicalResultSink ------hashAgg[LOCAL] --------PhysicalProject ----------NestedLoopJoin[CROSS_JOIN] -------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() ---------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() -----------------PhysicalProject -------------------PhysicalOlapScan[t1] +------------PhysicalProject +--------------hashJoin[LEFT_ANTI_JOIN] hashCondition=((t1.c1 = t3.c3)) otherCondition=() +----------------hashJoin[LEFT_SEMI_JOIN] hashCondition=((t1.c1 = alias2.c2)) otherCondition=() +------------------PhysicalProject +--------------------PhysicalOlapScan[t1] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------PhysicalProject +----------------------PhysicalOlapScan[t2] ----------------PhysicalDistribute[DistributionSpecHash] ------------------PhysicalProject ---------------------PhysicalOlapScan[t2] ---------------PhysicalDistribute[DistributionSpecHash] -----------------PhysicalProject -------------------PhysicalOlapScan[t3] +--------------------PhysicalOlapScan[t3] ------------PhysicalDistribute[DistributionSpecReplicated] ---------------PhysicalLimit[GLOBAL] -----------------PhysicalDistribute[DistributionSpecGather] -------------------PhysicalLimit[LOCAL] +--------------PhysicalProject +----------------PhysicalLimit[GLOBAL] +------------------PhysicalDistribute[DistributionSpecGather] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------PhysicalLimit[LOCAL] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[t3] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[t4] +--------------------------hashJoin[INNER_JOIN] hashCondition=((t3.c3 = t4.c4)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[t3] +----------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[t4] Hint log: Used: [shuffle]_2 diff --git a/regression-test/data/nereids_p0/hint/test_leading.out b/regression-test/data/nereids_p0/hint/test_leading.out index 07cc7c16284969..e7bca5b4023b5b 100644 --- a/regression-test/data/nereids_p0/hint/test_leading.out +++ b/regression-test/data/nereids_p0/hint/test_leading.out @@ -2162,44 +2162,6 @@ Used: [broadcast]_2 UnUsed: SyntaxError: --- !select90_3 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[INNER_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalOlapScan[t3] - -Hint log: -Used: [broadcast]_2 [shuffle]_3 -UnUsed: -SyntaxError: - --- !select90_5 -- -PhysicalResultSink ---hashAgg[GLOBAL] -----PhysicalDistribute[DistributionSpecGather] -------hashAgg[LOCAL] ---------hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalOlapScan[t2] -----------PhysicalDistribute[DistributionSpecHash] -------------PhysicalOlapScan[t3] - -Hint log: -Used: [broadcast]_2 [shuffle]_3 -UnUsed: -SyntaxError: - -- !select92_1 -- PhysicalResultSink --hashAgg[GLOBAL] @@ -2674,11 +2636,10 @@ PhysicalResultSink ----PhysicalDistribute[DistributionSpecGather] ------hashAgg[LOCAL] --------hashJoin[INNER_JOIN] hashCondition=((t2.c2 = t3.c3)) otherCondition=() -----------PhysicalDistribute[DistributionSpecHash] -------------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() ---------------PhysicalOlapScan[t1] ---------------PhysicalDistribute[DistributionSpecReplicated] -----------------PhysicalOlapScan[t2] +----------hashJoin[INNER_JOIN] hashCondition=((t1.c1 = t2.c2)) otherCondition=() +------------PhysicalOlapScan[t1] +------------PhysicalDistribute[DistributionSpecReplicated] +--------------PhysicalOlapScan[t2] ----------PhysicalDistribute[DistributionSpecHash] ------------PhysicalOlapScan[t3] 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/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_p0/hint/multi_leading.groovy b/regression-test/suites/nereids_p0/hint/multi_leading.groovy index 0e22c0f455a60f..c568165a059e84 100644 --- a/regression-test/suites/nereids_p0/hint/multi_leading.groovy +++ b/regression-test/suites/nereids_p0/hint/multi_leading.groovy @@ -80,7 +80,6 @@ suite("multi_leading") { } // test cte inline - qt_sql1_1 """explain shape plan with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" qt_sql1_2 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t1 cte)*/ count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" qt_sql1_3 """explain shape plan with cte as (select /*+ leading(t1 t2) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t1 cte)*/ count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" qt_sql1_4 """explain shape plan with cte as (select /*+ leading(t1 t2) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t1 cte)*/ count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" @@ -90,8 +89,7 @@ suite("multi_leading") { qt_sql1_res_3 """with cte as (select /*+ leading(t1 t2) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t1 cte)*/ count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" qt_sql1_res_4 """with cte as (select /*+ leading(t1 t2) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t1 cte)*/ count(*) from cte,t1 where cte.c1 = t1.c1 and t1.c1 > 300;""" - // test subquery alone - qt_sql2_1 """explain shape plan select count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" + // // test subquery alone qt_sql2_2 """explain shape plan select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql2_3 """explain shape plan select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql2_4 """explain shape plan select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" @@ -101,8 +99,7 @@ suite("multi_leading") { qt_sql2_res_3 """select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql2_res_4 """select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - // test subquery + cte - qt_sql3_1 """explain shape plan with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;""" + // // test subquery + cte qt_sql3_2 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_3 """explain shape plan with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_4 """explain shape plan with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" @@ -112,24 +109,23 @@ suite("multi_leading") { qt_sql3_res_3 """with cte as (select c11, c1 from t1 join t2 on c1 = c2) select count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" qt_sql3_res_4 """with cte as (select /*+ leading(t2 t1) */ c11, c1 from t1 join t2 on c1 = c2) select /*+ leading(t3 alias1 cte) */ count(*) from (select /*+ leading(t2 t1) */ c1, c11 from t1 join t2 on c1 = c2) as alias1 join t3 on alias1.c1 = t3.c3 join cte on alias1.c1 = cte.c11;;""" - // test multi level subqueries - qt_sql4_0 """explain shape plan select count(*) from (select c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" + // // test multi level subqueries qt_sql4_1 """explain shape plan select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql4_2 """explain shape plan select count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql4_3 """explain shape plan select count(*) from (select c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql4_4 """explain shape plan select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - explain { - sql """shape plan select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - contains("SyntaxError: leading(t4 t2) Msg:one query block can only have one leading clause") - } - explain { - sql """shape plan select count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - contains("SyntaxError: leading(t4 t2) Msg:one query block can only have one leading clause") - } - explain { - sql """shape plan select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - contains("UnUsed: leading(alias2 t1)") - } + // explain { + // sql """shape plan select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" + // contains("SyntaxError: leading(t4 t2) Msg:one query block can only have one leading clause") + // } + // explain { + // sql """shape plan select count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" + // contains("SyntaxError: leading(t4 t2) Msg:one query block can only have one leading clause") + // } + // explain { + // sql """shape plan select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" + // contains("UnUsed: leading(alias2 t1)") + // } qt_sql4_res_0 """select count(*) from (select c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql4_res_1 """select /*+ leading(t3 alias1) */ count(*) from (select c1, c11 from t1 join (select c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" @@ -140,7 +136,6 @@ suite("multi_leading") { qt_sql4_res_6 """select count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" qt_sql4_res_7 """select /*+ leading(t3 alias1) */ count(*) from (select /*+ leading(alias2 t1) */ c1, c11 from t1 join (select /*+ leading(t4 t2) */ c2, c22 from t2 join t4 on c2 = c4) as alias2 on c1 = alias2.c2) as alias1 join t3 on alias1.c1 = t3.c3;""" - // use cte in scalar query - qt_sql5_1 """explain shape plan with cte as (select c11, c1 from t1) SELECT c1 FROM cte group by c1 having sum(cte.c11) > (select 0.05 * avg(t1.c11) from t1 join cte on t1.c1 = cte.c11 )""" - qt_sql5_2 """explain shape plan with cte as (select c11, c1 from t1) SELECT c1 FROM cte group by c1 having sum(cte.c11) > (select /*+ leading(cte t1) */ 0.05 * avg(t1.c11) from t1 join cte on t1.c1 = cte.c11 )""" + // // use cte in scalar query + qt_sql5_2 """with cte as (select c11, c1 from t1) SELECT count(c1) FROM cte group by c1 having sum(cte.c11) > (select /*+ leading(cte t1) */ 0.05 * avg(t1.c11) from t1 join cte on t1.c1 = cte.c11 ) order by c1 limit 5""" } diff --git a/regression-test/suites/nereids_p0/hint/test_use_mv.groovy b/regression-test/suites/nereids_p0/hint/test_use_mv.groovy new file mode 100644 index 00000000000000..3885dcf3306fbe --- /dev/null +++ b/regression-test/suites/nereids_p0/hint/test_use_mv.groovy @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +suite("test_use_mv") { + // create database and tables + sql 'DROP DATABASE IF EXISTS test_use_mv' + sql 'CREATE DATABASE IF NOT EXISTS test_use_mv' + sql 'use test_use_mv' + + // setting planner to nereids + sql 'set exec_mem_limit=21G' + sql 'set be_number_for_test=1' + sql 'set parallel_pipeline_task_num=1' + sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" + sql 'set enable_nereids_planner=true' + sql "set ignore_shape_nodes='PhysicalProject'" + sql 'set enable_fallback_to_original_planner=false' + sql 'set runtime_filter_mode=OFF' + + sql """drop table if exists t1;""" + // create tables + sql """ + CREATE TABLE `t1` ( + `k1` int(11) NULL, + `k2` int(11) NULL, + `v1` int(11) SUM NULL + ) ENGINE=OLAP + AGGREGATE KEY(`k1`, `k2`) + COMMENT 'OLAP' + DISTRIBUTED BY HASH(`k1`) BUCKETS 3 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "in_memory" = "false", + "storage_format" = "V2", + "disable_auto_compaction" = "false" + ); + """ + sql """ alter table t1 add rollup r1(k2, k1); """ + waitForRollUpJob("t1", 5000, 1) + sql """ alter table t1 add rollup r2(k2); """ + waitForRollUpJob("t1", 5000, 1) + createMV("create materialized view k1_k2_sumk3 as select k1, k2, sum(v1) from t1 group by k1, k2;") + explain { + sql """select /*+ no_use_mv */ k1 from t1;""" + notContains("t1(r1)") + } + explain { + sql """select /*+ no_use_mv(t1) */ k1 from t1;""" + contains("parameter of no_use_mv hint must be in pairs") + } + explain { + sql """select /*+ no_use_mv(t1.`*`) */ k1 from t1;""" + contains("t1(t1)") + } + explain { + sql """select /*+ use_mv(t1.`*`) */ k1 from t1;""" + contains("use_mv hint should only have one mv in one table") + } + explain { + sql """select /*+ use_mv(t1.r1,t1.r2) */ k1 from t1;""" + contains("use_mv hint should only have one mv in one table") + } + explain { + sql """select /*+ use_mv(t1.r1) use_mv(t1.r2) */ k1 from t1;""" + contains("one use_mv hint is allowed") + } + explain { + sql """select /*+ no_use_mv(t1.r1) no_use_mv(t1.r2) */ k1 from t1;""" + contains("only one no_use_mv hint is allowed") + } + explain { + sql """select /*+ no_use_mv(t1.r3) */ k1 from t1;""" + contains("do not have mv: r3 in table: t1") + } + explain { + sql """select /*+ use_mv(t1.r1) no_use_mv(t1.r1) */ k1 from t1;""" + contains("conflict mv exist in use_mv and no_use_mv in the same time") + } + explain { + sql """select /*+ use_mv(t1.k1_k2_sumk3) */ k1, k2, sum(v1) from t1 group by k1, k2;""" + contains("t1(k1_k2_sumk3)") + } + explain { + sql """select /*+ use_mv(t1.k1_k2_sumk3) */ k1, k2, min(v1) from t1 group by k1, k2;""" + notContains("t1(k1_k2_sumk3)") + } + +} 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/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