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 82e03fe4b16cff..2e4fc59a5c27e1 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 @@ -29,7 +29,6 @@ import org.apache.doris.nereids.rules.analysis.LogicalSubQueryAliasToLogicalProject; import org.apache.doris.nereids.rules.analysis.NormalizeAggregate; import org.apache.doris.nereids.rules.expression.CheckLegalityAfterRewrite; -import org.apache.doris.nereids.rules.expression.ExpressionNormalization; import org.apache.doris.nereids.rules.expression.ExpressionNormalizationAndOptimization; import org.apache.doris.nereids.rules.expression.ExpressionRewrite; import org.apache.doris.nereids.rules.expression.QueryColumnCollector; @@ -294,6 +293,21 @@ public class Rewriter extends AbstractBatchJobExecutor { topDown(new ConvertInnerOrCrossJoin()), topDown(new ProjectOtherJoinConditionForNestedLoopJoin()) ), + topic("Set operation optimization", + // Do MergeSetOperation first because we hope to match pattern of Distinct SetOperator. + topDown(new PushProjectThroughUnion(), new MergeProjects()), + bottomUp(new MergeSetOperations(), new MergeSetOperationsExcept()), + bottomUp(new PushProjectIntoOneRowRelation()), + topDown(new MergeOneRowRelationIntoUnion()), + costBased(topDown(new InferSetOperatorDistinct())), + topDown(new BuildAggForUnion()), + bottomUp(new EliminateEmptyRelation()), + // when union has empty relation child and constantExprsList is not empty, + // after EliminateEmptyRelation, project can be pushed into union + topDown(new PushProjectIntoUnion()) + ), + // putting the "Column pruning and infer predicate" topic behind the "Set operation optimization" + // is because that pulling up predicates from union needs EliminateEmptyRelation in union child topic("Column pruning and infer predicate", custom(RuleType.COLUMN_PRUNING, ColumnPruning::new), custom(RuleType.INFER_PREDICATES, InferPredicates::new), @@ -307,24 +321,11 @@ public class Rewriter extends AbstractBatchJobExecutor { // after eliminate outer join, we can move some filters to join.otherJoinConjuncts, // this can help to translate plan to backend topDown(new PushFilterInsideJoin()), - topDown(new FindHashConditionForJoin()), - topDown(new ExpressionNormalization()) + topDown(new FindHashConditionForJoin()) ), - // this rule should invoke after ColumnPruning custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new), - topic("Set operation optimization", - // Do MergeSetOperation first because we hope to match pattern of Distinct SetOperator. - topDown(new PushProjectThroughUnion(), new MergeProjects()), - bottomUp(new MergeSetOperations(), new MergeSetOperationsExcept()), - bottomUp(new PushProjectIntoOneRowRelation()), - topDown(new MergeOneRowRelationIntoUnion()), - topDown(new PushProjectIntoUnion()), - costBased(topDown(new InferSetOperatorDistinct())), - topDown(new BuildAggForUnion()) - ), - topic("Eliminate GroupBy", topDown(new EliminateGroupBy(), new MergeAggregate(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index 26868665b10806..66176561255519 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -47,6 +47,7 @@ import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterScanRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectScanRule; +import org.apache.doris.nereids.rules.expression.ExpressionNormalization; import org.apache.doris.nereids.rules.expression.ExpressionOptimization; import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; @@ -88,6 +89,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; import org.apache.doris.nereids.rules.rewrite.ConvertOuterJoinToAntiJoin; import org.apache.doris.nereids.rules.rewrite.CreatePartitionTopNFromWindow; +import org.apache.doris.nereids.rules.rewrite.EliminateFilter; import org.apache.doris.nereids.rules.rewrite.EliminateOuterJoin; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.MergeGenerates; @@ -153,7 +155,12 @@ public class RuleSet { new PushDownAliasThroughJoin(), new PushDownFilterThroughWindow(), new PushDownFilterThroughPartitionTopN(), - new ExpressionOptimization() + new ExpressionOptimization(), + // some useless predicates(e.g. 1=1) can be inferred by InferPredicates, + // the FoldConstantRule in ExpressionNormalization can fold 1=1 to true + // and EliminateFilter can eliminate the useless filter + new ExpressionNormalization(), + new EliminateFilter() ); public static final List IMPLEMENTATION_RULES = planRuleFactories() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateEmptyRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateEmptyRelation.java index 204499fd9b960c..c45d55bb8ef2d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateEmptyRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateEmptyRelation.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.rules.rewrite; +import org.apache.doris.nereids.annotation.DependsRules; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.UnaryNode; @@ -45,6 +46,9 @@ /** * try to eliminate sub plan tree which contains EmptyRelation */ +@DependsRules ({ + BuildAggForUnion.class +}) public class EliminateEmptyRelation implements RewriteRuleFactory { @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java index 04db263c3655dc..5256c7744b9837 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InferPredicates.java @@ -19,17 +19,23 @@ import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.PlanUtils; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Sets; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -107,6 +113,45 @@ public Plan visitLogicalFilter(LogicalFilter filter, JobContext return filter; } + @Override + public Plan visitLogicalExcept(LogicalExcept except, JobContext context) { + except = visitChildren(this, except, context); + Set baseExpressions = pullUpPredicates(except); + if (baseExpressions.isEmpty()) { + return except; + } + ImmutableList.Builder builder = ImmutableList.builder(); + builder.add(except.child(0)); + for (int i = 1; i < except.arity(); ++i) { + Map replaceMap = new HashMap<>(); + for (int j = 0; j < except.getOutput().size(); ++j) { + NamedExpression output = except.getOutput().get(j); + replaceMap.put(output, except.getRegularChildOutput(i).get(j)); + } + builder.add(inferNewPredicate(except.child(i), ExpressionUtils.replace(baseExpressions, replaceMap))); + } + return except.withChildren(builder.build()); + } + + @Override + public Plan visitLogicalIntersect(LogicalIntersect intersect, JobContext context) { + intersect = visitChildren(this, intersect, context); + Set baseExpressions = pullUpPredicates(intersect); + if (baseExpressions.isEmpty()) { + return intersect; + } + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = 0; i < intersect.arity(); ++i) { + Map replaceMap = new HashMap<>(); + for (int j = 0; j < intersect.getOutput().size(); ++j) { + NamedExpression output = intersect.getOutput().get(j); + replaceMap.put(output, intersect.getRegularChildOutput(i).get(j)); + } + builder.add(inferNewPredicate(intersect.child(i), ExpressionUtils.replace(baseExpressions, replaceMap))); + } + return intersect.withChildren(builder.build()); + } + private Set getAllExpressions(Plan left, Plan right, Optional condition) { Set baseExpressions = pullUpPredicates(left); baseExpressions.addAll(pullUpPredicates(right)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java index dec288a6f52d4c..8082c0624a6047 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpPredicates.java @@ -20,15 +20,22 @@ import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.InPredicate; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.ExpressionUtils; @@ -38,6 +45,8 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.util.HashMap; +import java.util.HashSet; import java.util.IdentityHashMap; import java.util.List; import java.util.Map; @@ -60,6 +69,78 @@ public ImmutableSet visit(Plan plan, Void context) { return ImmutableSet.of(); } + @Override + public ImmutableSet visitLogicalOneRowRelation(LogicalOneRowRelation r, Void context) { + ImmutableSet.Builder predicates = ImmutableSet.builder(); + for (NamedExpression expr : r.getProjects()) { + if (expr instanceof Alias && expr.child(0) instanceof Literal) { + predicates.add(new EqualTo(expr.toSlot(), expr.child(0))); + } + } + return predicates.build(); + } + + @Override + public ImmutableSet visitLogicalIntersect(LogicalIntersect intersect, Void context) { + return cacheOrElse(intersect, () -> { + ImmutableSet.Builder builder = ImmutableSet.builder(); + for (int i = 0; i < intersect.children().size(); ++i) { + Plan child = intersect.child(i); + Set childFilters = child.accept(this, context); + if (childFilters.isEmpty()) { + continue; + } + Map replaceMap = new HashMap<>(); + for (int j = 0; j < intersect.getOutput().size(); ++j) { + NamedExpression output = intersect.getOutput().get(j); + replaceMap.put(intersect.getRegularChildOutput(i).get(j), output); + } + builder.addAll(ExpressionUtils.replace(childFilters, replaceMap)); + } + return getAvailableExpressions(builder.build(), intersect); + }); + } + + @Override + public ImmutableSet visitLogicalExcept(LogicalExcept except, Void context) { + return cacheOrElse(except, () -> { + if (except.arity() < 1) { + return ImmutableSet.of(); + } + Set firstChildFilters = except.child(0).accept(this, context); + if (firstChildFilters.isEmpty()) { + return ImmutableSet.of(); + } + Map replaceMap = new HashMap<>(); + for (int i = 0; i < except.getOutput().size(); ++i) { + NamedExpression output = except.getOutput().get(i); + replaceMap.put(except.getRegularChildOutput(0).get(i), output); + } + return ImmutableSet.copyOf(ExpressionUtils.replace(firstChildFilters, replaceMap)); + }); + } + + @Override + public ImmutableSet visitLogicalUnion(LogicalUnion union, Void context) { + return cacheOrElse(union, () -> { + if (!union.getConstantExprsList().isEmpty() && union.arity() == 0) { + return getFiltersFromUnionConstExprs(union); + } else if (union.getConstantExprsList().isEmpty() && union.arity() != 0) { + return getFiltersFromUnionChild(union, context); + } else if (!union.getConstantExprsList().isEmpty() && union.arity() != 0) { + HashSet fromChildFilters = new HashSet<>(getFiltersFromUnionChild(union, context)); + if (fromChildFilters.isEmpty()) { + return ImmutableSet.of(); + } + if (!ExpressionUtils.unionConstExprsSatisfyConjuncts(union, fromChildFilters)) { + return ImmutableSet.of(); + } + return ImmutableSet.copyOf(fromChildFilters); + } + return ImmutableSet.of(); + }); + } + @Override public ImmutableSet visitLogicalFilter(LogicalFilter filter, Void context) { return cacheOrElse(filter, () -> { @@ -77,6 +158,10 @@ public ImmutableSet visitLogicalJoin(LogicalJoin rightPredicates = join.right().accept(this, context); predicates.addAll(leftPredicates); predicates.addAll(rightPredicates); + if (join.getJoinType() == JoinType.CROSS_JOIN || join.getJoinType() == JoinType.INNER_JOIN) { + predicates.addAll(join.getHashJoinConjuncts()); + predicates.addAll(join.getOtherJoinConjuncts()); + } return getAvailableExpressions(predicates, join); }); } @@ -138,6 +223,9 @@ private ImmutableSet cacheOrElse(Plan plan, Supplier getAvailableExpressions(Set predicates, Plan plan) { + if (predicates.isEmpty()) { + return ImmutableSet.of(); + } Set inferPredicates = PredicatePropagation.infer(predicates); Builder newPredicates = ImmutableSet.builderWithExpectedSize(predicates.size() + 10); Set outputSet = plan.getOutputSet(); @@ -159,4 +247,55 @@ private ImmutableSet getAvailableExpressions(Set predica private boolean hasAgg(Expression expression) { return expression.anyMatch(AggregateFunction.class::isInstance); } + + private ImmutableSet getFiltersFromUnionChild(LogicalUnion union, Void context) { + Set filters = new HashSet<>(); + for (int i = 0; i < union.getArity(); ++i) { + Plan child = union.child(i); + Set childFilters = child.accept(this, context); + if (childFilters.isEmpty()) { + return ImmutableSet.of(); + } + Map replaceMap = new HashMap<>(); + for (int j = 0; j < union.getOutput().size(); ++j) { + NamedExpression output = union.getOutput().get(j); + replaceMap.put(union.getRegularChildOutput(i).get(j), output); + } + Set unionFilters = ExpressionUtils.replace(childFilters, replaceMap); + if (0 == i) { + filters.addAll(unionFilters); + } else { + filters.retainAll(unionFilters); + } + if (filters.isEmpty()) { + return ImmutableSet.of(); + } + } + return ImmutableSet.copyOf(filters); + } + + private ImmutableSet getFiltersFromUnionConstExprs(LogicalUnion union) { + List> constExprs = union.getConstantExprsList(); + ImmutableSet.Builder filtersFromConstExprs = ImmutableSet.builder(); + for (int col = 0; col < union.getOutput().size(); ++col) { + Expression compareExpr = union.getOutput().get(col); + Set options = new HashSet<>(); + for (List constExpr : constExprs) { + if (constExpr.get(col) instanceof Alias + && ((Alias) constExpr.get(col)).child() instanceof Literal) { + options.add(((Alias) constExpr.get(col)).child()); + } else { + options.clear(); + break; + } + } + options.removeIf(option -> option instanceof NullLiteral); + if (options.size() > 1) { + filtersFromConstExprs.add(new InPredicate(compareExpr, options)); + } else if (options.size() == 1) { + filtersFromConstExprs.add(new EqualTo(compareExpr, options.iterator().next())); + } + } + return filtersFromConstExprs.build(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoUnion.java index 130d4b04d3f2fa..31ecb10925fdb0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoUnion.java @@ -19,11 +19,14 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.util.ExpressionUtils; @@ -42,7 +45,9 @@ public Rule build() { return logicalProject(logicalUnion() .when(u -> u.getQualifier() == Qualifier.ALL) .when(u -> u.arity() == 0) - ).then(p -> { + ).thenApply(ctx -> { + LogicalProject p = ctx.root; + ExpressionRewriteContext expressionRewriteContext = new ExpressionRewriteContext(ctx.cascadesContext); LogicalUnion union = p.child(); ImmutableList.Builder> newConstExprs = ImmutableList.builder(); for (List constExprs : union.getConstantExprsList()) { @@ -60,9 +65,11 @@ public Rule build() { ImmutableList.Builder newProjections = ImmutableList.builder(); for (NamedExpression old : p.getProjects()) { if (old instanceof SlotReference) { - newProjections.add(replaceRootMap.get(old)); + newProjections.add((NamedExpression) FoldConstantRule.evaluate(replaceRootMap.get(old), + expressionRewriteContext)); } else { - newProjections.add(ExpressionUtils.replaceNameExpression(old, replaceMap)); + newProjections.add((NamedExpression) FoldConstantRule.evaluate( + ExpressionUtils.replaceNameExpression(old, replaceMap), expressionRewriteContext)); } } newConstExprs.add(newProjections.build()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 2f238b68757613..88ab5bba479e4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -22,6 +22,7 @@ import org.apache.doris.common.NereidsException; import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule; import org.apache.doris.nereids.trees.TreeNode; @@ -52,9 +53,11 @@ import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.coercion.NumericType; +import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -921,4 +924,30 @@ public static List filter(List e } return result.build(); } + + /** test whether unionConstExprs satisfy conjuncts */ + public static boolean unionConstExprsSatisfyConjuncts(LogicalUnion union, Set conjuncts) { + CascadesContext tempCascadeContext = CascadesContext.initContext( + ConnectContext.get().getStatementContext(), union, PhysicalProperties.ANY); + ExpressionRewriteContext rewriteContext = new ExpressionRewriteContext(tempCascadeContext); + for (List constOutput : union.getConstantExprsList()) { + Map replaceMap = new HashMap<>(); + for (int i = 0; i < constOutput.size(); i++) { + Expression output = constOutput.get(i); + if (output instanceof Alias) { + replaceMap.put(union.getOutput().get(i), ((Alias) output).child()); + } else { + replaceMap.put(union.getOutput().get(i), output); + } + } + for (Expression conjunct : conjuncts) { + Expression res = FoldConstantRule.evaluate(ExpressionUtils.replace(conjunct, replaceMap), + rewriteContext); + if (!res.equals(BooleanLiteral.TRUE)) { + return false; + } + } + } + return true; + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java index f79bdff9ec5405..95e676e814509b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java @@ -75,6 +75,7 @@ protected void runBeforeAll() throws Exception { connectContext.setDatabase("test"); connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + connectContext.getSessionVariable().setEnableFoldConstantByBe(true); } @Test @@ -609,7 +610,7 @@ void inferPredicatesTest22() { void innerJoinShouldNotInferUnderLeftJoinOnClausePredicates() { String sql = "select * from student s1" + " left join (select sid as id1, sid as id2, grade from score) s2 on s1.id = s2.id1 and s1.id = 1" - + " join (select sid as id1, sid as id2, grade from score) s3 on s1.id = s3.id1 where s1.id = 2"; + + " join (select sid as id1, sid as id2, grade from score) s3 on s1.id = s3.id1 where s1.id =2"; PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); PlanChecker.from(connectContext) .analyze(sql) @@ -645,4 +646,132 @@ void inferPredicateByConstValue() { )) ); } + + @Test + void pullUpPredicateFromIntersect() { + String sql = "select c1 from (select age c1,id from student where id <10 intersect select age,id from student where id >1) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 2 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("(sid > 1)") + && filter.getPredicate().toSql().contains("(sid < 10)")) + ); + } + + @Test + void pullUpPredicateFromExcept() { + String sql = "select c1 from (select age c1,id from student where id <10 except select age,id from student where id >1) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 1 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid < 10")) + ); + } + + @Test + void pullUpPredicateFromUnion() { + String sql = "select c1 from (select 2 c1,id from course where id <10 union select age,id from student where id <10) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 1 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid < 10")) + ); + + String sql2 = "select c1 from (select 2 c1,id from course where id <10 union all select age,id from student where id <10) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql2).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql2) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 1 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid < 10")) + ); + } + + @Test + void pullUpPredicateFromUnionConst() { + String sql = "select c2 from (select 2 id,'abc' c2 union all select 1 id,'abbbb' c4 ) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 1 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid IN (1, 2)")) + ); + + String sql2 = "select id,t2.sid from (select 2 id,'abc' b from score limit 0 offset 0 union all select 1 id,'abb' c4) t inner join score t2 on t.id=t2.sid"; + PlanChecker.from(connectContext).analyze(sql2).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql2) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 1 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid = 1")) + ); + } + + @Test + void pullUpPredicateFromUnionConstAndChild() { + String sql = "select c2 from (select 2 id,4 c2 union all select age,4 from student where age>0) t inner join score t2 on t.id=t2.sid and t.c2=t2.cid"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 2 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid > 0") + && filter.getPredicate().toSql().contains("cid = 4")) + ); + + String sql2 = "select c2 from (select 2 id,4 c2 union all select age,id from student where age=2 and id <9) t inner join score t2 on t.id=t2.sid and t.c2=t2.cid"; + PlanChecker.from(connectContext).analyze(sql2).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql2) + .rewrite() + .matches(logicalFilter(logicalOlapScan()) + .when(filter -> filter.getConjuncts().size() == 2 + && ExpressionUtils.isInferred(filter.getPredicate()) + && filter.getPredicate().toSql().contains("sid = 2") + && filter.getPredicate().toSql().contains("cid < 9")) + ); + } + + @Test + void inferPredicateFromIntersect() { + String sql = "select age c1,id from student where id <10 intersect select age,id from student where id >1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalIntersect(logicalProject(logicalFilter().when(filter -> filter.getConjuncts().size() == 2)), + logicalProject(logicalFilter().when(filter -> filter.getConjuncts().size() == 2)))); + } + + @Test + void inferPredicateFromExcept() { + String sql = "select age c1,id from student where id <10 except select age,id from student where id >1"; + PlanChecker.from(connectContext).analyze(sql).rewrite().printlnTree(); + PlanChecker.from(connectContext) + .analyze(sql) + .rewrite() + .matches(logicalFilter(logicalOlapScan().when(scan -> scan.getTable().getName().equals("student"))) + .when(filter -> filter.getConjuncts().size() == 2)); + } } diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query8.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query8.out index a93e8197071a0b..383c334a67fa74 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query8.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out index 5d80a10fad25e9..ec63cd6aabdc4f 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_down_expression_in_hash_join.out @@ -64,7 +64,7 @@ PhysicalResultSink -- !pushdown_left_semi_join_subquery -- PhysicalResultSink ---hashJoin[LEFT_SEMI_JOIN] hashCondition=((expr_(id + 1) = expr_cast(id as BIGINT))) otherCondition=() +--hashJoin[LEFT_SEMI_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() ----PhysicalOlapScan[t1] ----PhysicalOlapScan[t2] @@ -78,7 +78,7 @@ PhysicalResultSink -- !pushdown_left_anti_join_subquery -- PhysicalResultSink ---hashJoin[NULL_AWARE_LEFT_ANTI_JOIN] hashCondition=((expr_(id + 1) = expr_cast(id as BIGINT))) otherCondition=() +--hashJoin[NULL_AWARE_LEFT_ANTI_JOIN] hashCondition=((expr_cast(id as BIGINT) = expr_(cast(id as BIGINT) - 1))) otherCondition=() ----PhysicalOlapScan[t1] ----PhysicalOlapScan[t2] diff --git a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out index 1330d3350cd397..906d59fa9016dc 100644 --- a/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out +++ b/regression-test/data/nereids_rules_p0/filter_push_down/push_filter_through.out @@ -331,7 +331,8 @@ PhysicalResultSink --------PhysicalOlapScan[t1] ------filter((t2.id = 2)) --------PhysicalOlapScan[t2] -----PhysicalOlapScan[t3] +----filter((t3.id = 2)) +------PhysicalOlapScan[t3] -- !push_filter_subquery -- PhysicalResultSink @@ -341,7 +342,8 @@ PhysicalResultSink --------PhysicalOlapScan[t1] ------filter((t2.id = 2)) --------PhysicalOlapScan[t2] -----PhysicalOlapScan[t3] +----filter((t3.id = 2)) +------PhysicalOlapScan[t3] -- !filter_window_row_number -- PhysicalResultSink diff --git a/regression-test/data/nereids_rules_p0/infer_predicate/infer_intersect_except.out b/regression-test/data/nereids_rules_p0/infer_predicate/infer_intersect_except.out new file mode 100644 index 00000000000000..783f83efe61753 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/infer_predicate/infer_intersect_except.out @@ -0,0 +1,155 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !except -- +PhysicalResultSink +--PhysicalExcept +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((infer_intersect_except2.a > 0) and (infer_intersect_except2.b > 'ab')) +------PhysicalOlapScan[infer_intersect_except2] +----filter((infer_intersect_except2.a < 10) and (infer_intersect_except2.a > 0)) +------PhysicalOlapScan[infer_intersect_except2] + +-- !except_to_empty -- +PhysicalResultSink +--PhysicalExcept +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((infer_intersect_except2.a > 0) and (infer_intersect_except2.b > 'ab')) +------PhysicalOlapScan[infer_intersect_except2] + +-- !except_not_infer_1_greater_than_0 -- +PhysicalResultSink +--PhysicalExcept +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((infer_intersect_except2.b > 'ab')) +------PhysicalOlapScan[infer_intersect_except2] + +-- !except_number_and_string -- +PhysicalResultSink +--hashAgg[GLOBAL] +----hashAgg[LOCAL] +------filter((infer_intersect_except1.a > 0)) +--------PhysicalOlapScan[infer_intersect_except1] + +-- !intersect -- +PhysicalResultSink +--PhysicalIntersect +----filter((infer_intersect_except1.a > 0) and (infer_intersect_except1.b > 'ab')) +------PhysicalOlapScan[infer_intersect_except1] +----filter((infer_intersect_except2.a > 0) and (infer_intersect_except2.b > 'ab')) +------PhysicalOlapScan[infer_intersect_except2] + +-- !intersect_empty -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !intersect_expr -- +PhysicalResultSink +--PhysicalIntersect +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((cast(a as BIGINT) < -1)) +------PhysicalOlapScan[infer_intersect_except2] + +-- !except_and_intersect -- +PhysicalResultSink +--PhysicalExcept +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----PhysicalIntersect +------filter((infer_intersect_except3.a = 1) and (infer_intersect_except3.b = 'abc')) +--------PhysicalOlapScan[infer_intersect_except3] +------filter((infer_intersect_except2.b > 'ab')) +--------PhysicalOlapScan[infer_intersect_except2] + +-- !except_and_intersect_except_predicate_to_right -- +PhysicalResultSink +--PhysicalExcept +----filter((infer_intersect_except1.a > 0)) +------PhysicalOlapScan[infer_intersect_except1] +----PhysicalIntersect +------filter((infer_intersect_except2.a < 10) and (infer_intersect_except2.a > 0) and (infer_intersect_except2.b > 'ab')) +--------PhysicalOlapScan[infer_intersect_except2] +------filter((infer_intersect_except3.a < 10) and (infer_intersect_except3.a > 0) and (infer_intersect_except3.b = 'abc')) +--------PhysicalOlapScan[infer_intersect_except3] + +-- !intersect_and_except -- +PhysicalResultSink +--PhysicalExcept +----PhysicalIntersect +------filter((infer_intersect_except1.a = 1) and (infer_intersect_except1.b = 'abc')) +--------PhysicalOlapScan[infer_intersect_except1] +------filter((infer_intersect_except2.b > 'ab')) +--------PhysicalOlapScan[infer_intersect_except2] +----filter((infer_intersect_except3.a = 1) and (infer_intersect_except3.b = 'abc')) +------PhysicalOlapScan[infer_intersect_except3] + +-- !function_intersect -- +PhysicalResultSink +--PhysicalIntersect +----filter((abs(a) < 3)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((abs(a) < 3)) +------PhysicalOlapScan[infer_intersect_except2] + +-- !function_except -- +PhysicalResultSink +--PhysicalExcept +----filter((abs(a) < 3)) +------PhysicalOlapScan[infer_intersect_except1] +----filter((abs(a) < 3)) +------PhysicalOlapScan[infer_intersect_except2] + +-- !except_res -- + +-- !except_to_empty_res -- + +-- !except_not_infer_1_greater_than_0_res -- +1 d2 + +-- !except_number_and_string_res -- +1 2 + +-- !intersect_res -- +1 d2 + +-- !intersect_empty_res -- + +-- !intersect_expr_res -- + +-- !except_and_intersect_res -- +1 d2 + +-- !except_and_intersect_except_predicate_to_right_res -- +1 d2 + +-- !intersect_and_except_res -- + +-- !function_intersect_res -- +0 +1 + +-- !function_except_res -- +2 + +-- !different_type_date_string -- +PhysicalResultSink +--PhysicalIntersect +----filter((infer_intersect_except4.d_datetimev2 > '2020-01-01 00:00:00')) +------PhysicalOlapScan[infer_intersect_except4] +----filter((infer_intersect_except4.d_char100 < 'abc')) +------PhysicalOlapScan[infer_intersect_except4] + +-- !different_type_int_string -- +PhysicalResultSink +--PhysicalIntersect +----filter((infer_intersect_except4.d_int > 2)) +------PhysicalOlapScan[infer_intersect_except4] +----filter((infer_intersect_except4.d_char100 < 'abc')) +------PhysicalOlapScan[infer_intersect_except4] + +-- !different_type_date_string_res -- + +-- !different_type_int_string_res -- + diff --git a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out index bb0367626de8b0..524559cabeb34d 100644 --- a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out +++ b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.out @@ -1,11 +1,18 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !test_pull_up_literal -- +-- !test_pull_up_literal1 -- PhysicalResultSink --PhysicalProject ----hashJoin[INNER_JOIN] hashCondition=((col1 = ds.col1) and (col2 = ds.col2)) otherCondition=() -------PhysicalProject ---------PhysicalOneRowRelation -------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def') and (ds.col2 = ds.col2)) +------PhysicalOneRowRelation +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) +--------PhysicalOlapScan[test_pull_up_predicate_literal] + +-- !test_pull_up_literal2 -- +PhysicalResultSink +--PhysicalProject +----hashJoin[INNER_JOIN] hashCondition=((col1 = ds.col1) and (col2 = ds.col2)) otherCondition=() +------PhysicalOneRowRelation +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) --------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_suquery -- @@ -14,7 +21,7 @@ PhysicalResultSink ----hashJoin[INNER_JOIN] hashCondition=((tmp.col1 = ds.col1) and (tmp.col2 = ds.col2)) otherCondition=() ------PhysicalProject --------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def') and (ds.col2 = ds.col2)) +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) --------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_extra_literal -- @@ -23,7 +30,7 @@ PhysicalResultSink ----hashJoin[INNER_JOIN] hashCondition=((tmp.col1 = ds.col1) and (tmp.col2 = ds.col2)) otherCondition=() ------PhysicalProject --------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def') and (ds.col2 = ds.col2)) +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) --------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_with_agg_func -- @@ -34,7 +41,7 @@ PhysicalResultSink --------hashAgg[GLOBAL] ----------hashAgg[LOCAL] ------------PhysicalOlapScan[test_pull_up_predicate_literal] -------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def') and (ds.col2 = ds.col2)) +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) --------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_to_empty_relation -- @@ -48,7 +55,7 @@ PhysicalResultSink ------PhysicalProject --------filter((t.col2 = 'def')) ----------PhysicalOlapScan[test_pull_up_predicate_literal] -------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def')) +------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) --------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_outer_has_agg -- @@ -59,7 +66,7 @@ PhysicalResultSink --------hashJoin[INNER_JOIN] hashCondition=((tmp.col1 = ds.col1) and (tmp.col2 = ds.col2)) otherCondition=() ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -----------filter((ds.col1 = 'abc') and (ds.col1 = ds.col1) and (ds.col2 = 'def') and (ds.col2 = ds.col2)) +----------filter((ds.col1 = 'abc') and (ds.col2 = 'def')) ------------PhysicalOlapScan[test_pull_up_predicate_literal] -- !test_pull_up_literal_multi_join -- @@ -154,7 +161,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type6 -- PhysicalResultSink @@ -165,7 +173,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type7 -- PhysicalResultSink @@ -176,7 +185,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type8 -- PhysicalResultSink @@ -187,7 +197,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 127.0000000000000000000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type9 -- PhysicalResultSink @@ -198,7 +209,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_date = '2000-01-27')) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type10 -- PhysicalResultSink @@ -209,7 +221,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_datetime = '2000-01-27 00:00:00')) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type11 -- PhysicalResultSink @@ -220,7 +233,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type12 -- PhysicalResultSink @@ -231,7 +245,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type13 -- PhysicalResultSink @@ -242,7 +257,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 127.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type16 -- PhysicalResultSink @@ -313,7 +329,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type22 -- PhysicalResultSink @@ -324,7 +341,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type23 -- PhysicalResultSink @@ -335,7 +353,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type24 -- PhysicalResultSink @@ -346,29 +365,16 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 32767.0000000000000000000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type25 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_date = expr_cast(c1 as DATEV2))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type26 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_datetime = expr_cast(c1 as DATETIMEV2(0)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type27 -- PhysicalResultSink @@ -379,7 +385,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type28 -- PhysicalResultSink @@ -390,7 +397,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type29 -- PhysicalResultSink @@ -401,7 +409,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 32767.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type32 -- PhysicalResultSink @@ -472,7 +481,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type38 -- PhysicalResultSink @@ -483,7 +493,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type39 -- PhysicalResultSink @@ -494,7 +505,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type40 -- PhysicalResultSink @@ -505,29 +517,16 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 32768.0000000000000000000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type41 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_date = expr_cast(c1 as DATEV2))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type42 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_datetime = expr_cast(c1 as DATETIMEV2(0)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type43 -- PhysicalResultSink @@ -538,7 +537,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type44 -- PhysicalResultSink @@ -549,7 +549,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type45 -- PhysicalResultSink @@ -560,7 +561,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 32768.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type48 -- PhysicalResultSink @@ -631,7 +633,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type54 -- PhysicalResultSink @@ -642,7 +645,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type55 -- PhysicalResultSink @@ -653,7 +657,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type56 -- PhysicalResultSink @@ -664,29 +669,16 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_decimal as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type57 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_date = expr_cast(c1 as DATEV2))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type58 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_datetime = expr_cast(c1 as DATETIMEV2(0)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type59 -- PhysicalResultSink @@ -697,7 +689,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type60 -- PhysicalResultSink @@ -708,7 +701,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type61 -- PhysicalResultSink @@ -719,7 +713,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 2.14748364799E11)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type64 -- PhysicalResultSink @@ -790,7 +785,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type70 -- PhysicalResultSink @@ -801,7 +797,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type71 -- PhysicalResultSink @@ -812,7 +809,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type72 -- PhysicalResultSink @@ -823,7 +821,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_decimal as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type73 -- PhysicalResultSink @@ -834,7 +833,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type74 -- PhysicalResultSink @@ -845,7 +845,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_datetime as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type75 -- PhysicalResultSink @@ -856,7 +857,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type76 -- PhysicalResultSink @@ -867,7 +869,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type77 -- PhysicalResultSink @@ -878,7 +881,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 9.223372036854776E20)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type80 -- PhysicalResultSink @@ -889,7 +893,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_tinyint = 1)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type81 -- PhysicalResultSink @@ -900,7 +905,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_smallint = 1)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type82 -- PhysicalResultSink @@ -911,7 +917,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_int = 1)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type83 -- PhysicalResultSink @@ -922,7 +929,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_bigint = 1)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type84 -- PhysicalResultSink @@ -933,7 +941,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_largeint as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type85 -- PhysicalResultSink @@ -956,7 +965,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type87 -- PhysicalResultSink @@ -967,7 +977,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type88 -- PhysicalResultSink @@ -978,7 +989,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_decimal as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type89 -- PhysicalResultSink @@ -989,7 +1001,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type90 -- PhysicalResultSink @@ -1000,7 +1013,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_datetime as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type91 -- PhysicalResultSink @@ -1011,7 +1025,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type92 -- PhysicalResultSink @@ -1022,7 +1037,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type93 -- PhysicalResultSink @@ -1033,51 +1049,24 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 1.0)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type96 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DECIMALV3(7, 4)) = expr_cast(c1 as DECIMALV3(7, 4)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type97 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DECIMALV3(9, 4)) = expr_cast(c1 as DECIMALV3(9, 4)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type98 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DECIMALV3(14, 4)) = expr_cast(c1 as DECIMALV3(14, 4)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type99 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DECIMALV3(24, 4)) = expr_cast(c1 as DECIMALV3(24, 4)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type100 -- PhysicalResultSink @@ -1088,7 +1077,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_largeint as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type101 -- PhysicalResultSink @@ -1099,7 +1089,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type102 -- PhysicalResultSink @@ -1110,7 +1101,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type103 -- PhysicalResultSink @@ -1121,7 +1113,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type104 -- PhysicalResultSink @@ -1132,7 +1125,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 1.0001000000000000000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type105 -- PhysicalResultSink @@ -1143,7 +1137,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type106 -- PhysicalResultSink @@ -1154,7 +1149,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_datetime as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type107 -- PhysicalResultSink @@ -1165,7 +1161,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type108 -- PhysicalResultSink @@ -1176,7 +1173,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type109 -- PhysicalResultSink @@ -1187,95 +1185,72 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 1.0001)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type112 -- PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type113 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type114 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type115 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type116 -- +PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DECIMALV3(15, 12)) = expr_cast(c1 as DECIMALV3(15, 12)))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_largeint as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type113 -- +-- !const_value_and_join_column_type117 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DECIMALV3(17, 12)) = expr_cast(c1 as DECIMALV3(17, 12)))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type114 -- +-- !const_value_and_join_column_type118 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DECIMALV3(22, 12)) = expr_cast(c1 as DECIMALV3(22, 12)))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type115 -- +-- !const_value_and_join_column_type119 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DECIMALV3(32, 12)) = expr_cast(c1 as DECIMALV3(32, 12)))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type116 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type117 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type118 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type119 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type120 -- PhysicalResultSink @@ -1286,7 +1261,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 1.0000000000030000000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type121 -- PhysicalResultSink @@ -1297,7 +1273,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type122 -- PhysicalResultSink @@ -1308,7 +1285,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_datetime as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type123 -- PhysicalResultSink @@ -1319,7 +1297,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type124 -- PhysicalResultSink @@ -1330,7 +1309,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type125 -- PhysicalResultSink @@ -1341,7 +1321,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 1.000000000003)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type128 -- PhysicalResultSink @@ -1353,25 +1334,11 @@ PhysicalResultSink -- !const_value_and_join_column_type130 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DECIMALV3(26, 16)) = expr_cast(c1 as DECIMALV3(26, 16)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type131 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DECIMALV3(36, 16)) = expr_cast(c1 as DECIMALV3(36, 16)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type132 -- PhysicalResultSink @@ -1382,7 +1349,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_largeint as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type133 -- PhysicalResultSink @@ -1393,7 +1361,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_bool as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type134 -- PhysicalResultSink @@ -1404,7 +1373,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_float as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type135 -- PhysicalResultSink @@ -1415,7 +1385,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_double = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type136 -- PhysicalResultSink @@ -1426,7 +1397,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_decimal = 12232.2398272342335234000)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type137 -- PhysicalResultSink @@ -1437,7 +1409,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type138 -- PhysicalResultSink @@ -1448,7 +1421,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_datetime as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type139 -- PhysicalResultSink @@ -1459,7 +1433,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_char as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type140 -- PhysicalResultSink @@ -1470,7 +1445,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_varchar as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type141 -- PhysicalResultSink @@ -1481,106 +1457,44 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_string as DOUBLE) = 12232.239827234234)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type144 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type145 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type146 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type147 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type148 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type149 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type150 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type151 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type152 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type153 -- PhysicalResultSink @@ -1591,7 +1505,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_date = '2024-07-01')) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type154 -- PhysicalResultSink @@ -1602,7 +1517,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_datetime = '2024-07-01 00:00:00')) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type155 -- PhysicalResultSink @@ -1642,14 +1558,7 @@ PhysicalResultSink -- !const_value_and_join_column_type158 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type159 -- PhysicalResultSink @@ -1664,153 +1573,84 @@ PhysicalResultSink -- !const_value_and_join_column_type160 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type161 -- PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type162 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type163 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type164 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type165 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type166 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type167 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type168 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type169 -- +PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_date as DATETIMEV2(0)) = expr_cast(c1 as DATETIMEV2(0)))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((cast(d_date as DATETIMEV2(0)) = '2024-08-02 10:10:00')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type162 -- +-- !const_value_and_join_column_type170 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type171 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_char = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_char = '2024-08-02 10:10:00.123332')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type163 -- +-- !const_value_and_join_column_type172 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_varchar = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_varchar = '2024-08-02 10:10:00.123332')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type164 -- +-- !const_value_and_join_column_type173 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type165 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type166 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type167 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type168 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type169 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_date as DATETIMEV2(0)) = expr_cast(c1 as DATETIMEV2(0)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type170 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_datetime as DATETIMEV2(6)) = expr_cast(c1 as DATETIMEV2(6)))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type171 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_char = t.c1)) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------filter((t2.d_char = '2024-08-02 10:10:00.123332')) -----------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type172 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_varchar = t.c1)) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------filter((t2.d_varchar = '2024-08-02 10:10:00.123332')) -----------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type173 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_string = t.c1)) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_string = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject @@ -1821,14 +1661,7 @@ PhysicalResultSink -- !const_value_and_join_column_type174 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type175 -- PhysicalResultSink @@ -1843,102 +1676,39 @@ PhysicalResultSink -- !const_value_and_join_column_type176 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type177 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type178 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type179 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type180 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type181 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type182 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type183 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type184 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type187 -- PhysicalResultSink @@ -1978,14 +1748,7 @@ PhysicalResultSink -- !const_value_and_join_column_type190 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type191 -- PhysicalResultSink @@ -2000,242 +1763,84 @@ PhysicalResultSink -- !const_value_and_join_column_type192 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type193 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type194 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type195 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type196 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type197 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type198 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type199 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type200 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type203 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_char = t.c1)) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------filter((t2.d_char = 'dtestvarcharvarchar')) -----------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type204 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_varchar = t.c1)) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------filter((t2.d_varchar = 'dtestvarcharvarchar')) -----------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type205 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_string = t.c1)) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------filter((t2.d_string = 'dtestvarcharvarchar')) -----------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type206 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type207 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv6 = expr_cast(c1 as IPV6))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type208 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type209 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type210 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] - --- !const_value_and_join_column_type211 -- -PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +PhysicalResultSink +--PhysicalEmptyRelation --- !const_value_and_join_column_type212 -- +-- !const_value_and_join_column_type203 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_char = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_char = 'dtestvarcharvarchar')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type213 -- +-- !const_value_and_join_column_type204 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_varchar = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_varchar = 'dtestvarcharvarchar')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type214 -- +-- !const_value_and_join_column_type205 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_string = t.c1)) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_string = 'dtestvarcharvarchar')) +----------PhysicalOlapScan[test_types] --- !const_value_and_join_column_type215 -- +-- !const_value_and_join_column_type206 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type207 -- PhysicalResultSink --PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() +----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv6 = expr_cast(c1 as IPV6))) otherCondition=() ------PhysicalLimit[GLOBAL] --------PhysicalLimit[LOCAL] ----------PhysicalProject @@ -2243,16 +1848,41 @@ PhysicalResultSink ------PhysicalProject --------PhysicalOlapScan[test_types] +-- !const_value_and_join_column_type208 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type209 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type210 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type211 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type212 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type213 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type214 -- +PhysicalResultSink +--PhysicalEmptyRelation + +-- !const_value_and_join_column_type215 -- +PhysicalResultSink +--PhysicalEmptyRelation + -- !const_value_and_join_column_type216 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type219 -- PhysicalResultSink @@ -2292,14 +1922,7 @@ PhysicalResultSink -- !const_value_and_join_column_type222 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type223 -- PhysicalResultSink @@ -2314,102 +1937,39 @@ PhysicalResultSink -- !const_value_and_join_column_type224 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type225 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type226 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type227 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type228 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type229 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type230 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type231 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type232 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type235 -- PhysicalResultSink @@ -2456,7 +2016,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_ipv4 = 2130706433)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type239 -- PhysicalResultSink @@ -2471,102 +2032,39 @@ PhysicalResultSink -- !const_value_and_join_column_type240 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_tinyint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type241 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type242 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_int as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type243 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bigint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type244 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_largeint as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type245 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_bool as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type246 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_float as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type247 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_double = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type248 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_decimal as DOUBLE) = expr_cast(c1 as DOUBLE))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type251 -- PhysicalResultSink @@ -2606,14 +2104,7 @@ PhysicalResultSink -- !const_value_and_join_column_type254 -- PhysicalResultSink ---PhysicalProject -----hashJoin[INNER_JOIN] hashCondition=((t2.d_ipv4 = expr_cast(c1 as IPV4))) otherCondition=() -------PhysicalLimit[GLOBAL] ---------PhysicalLimit[LOCAL] -----------PhysicalProject -------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] -------PhysicalProject ---------PhysicalOlapScan[test_types] +--PhysicalEmptyRelation -- !const_value_and_join_column_type255 -- PhysicalResultSink @@ -2624,7 +2115,8 @@ PhysicalResultSink ----------PhysicalProject ------------PhysicalStorageLayerAggregate[test_pull_up_predicate_literal] ------PhysicalProject ---------PhysicalOlapScan[test_types] +--------filter((t2.d_ipv6 = ffff:ffff:ffff:ffff:ffff:ffff:ffff:ffff)) +----------PhysicalOlapScan[test_types] -- !const_value_and_join_column_type_res0 -- 127 diff --git a/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out new file mode 100644 index 00000000000000..18ef848c0e6ba7 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.out @@ -0,0 +1,563 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !intersect -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.a < 1) and (test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a < 1) and (t3.b > 'ab')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------filter((test_pull_up_predicate_set_op1.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a < 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a < 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_one_side_constant_one_side_column -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a = 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a = 1) and (t3.b > 'ab')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_one_side -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.b > 'ab')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_one_side_no_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except_first_no_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except_second_no_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------filter((test_pull_up_predicate_set_op1.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a > 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except_no_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_different_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +------filter((test_pull_up_predicate_set_op1.a < 2)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_one_side_filter -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_with_const -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +------filter((test_pull_up_predicate_set_op1.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(a IN (133333, 2) and b IN ('aa', 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const_tinyint_int -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = expr_cast(a as INT)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(a IN (2, 3) and b IN ('aa', 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const_empty_relation -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = expr_cast(a as INT)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter((t3.a = 2) and (t3.b = 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const2_has_cast_to_null_different_type -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(a as DOUBLE) = expr_cast(a as DOUBLE)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(b IN ('aa', 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const2_has_cast_to_null_different_type -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(a as DOUBLE) = expr_cast(a as DOUBLE)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(b IN ('12', 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_different_type_int_cast_to_char -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(a as DOUBLE) = expr_cast(a as DOUBLE)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(b IN ('12', 'dd')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const_char3_char2 -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = expr_cast(a as INT)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----filter(a IN (2, 3, 5)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_child_and_const_exprs -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter(a IN (1, 2)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter(a IN (1, 2)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter(a IN (1, 2)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_child_and_const_exprs_andpredicates -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter(a IN (1, 2) and b IN ('2d', '3')) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----filter(a IN (1, 2) and b IN ('2d', '3')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_child_and_const_exprs_orpredicates -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((a IN (1, 2) OR b IN ('2d', '3'))) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_one_side_constant_one_side_column_left_join -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a = 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a = 1) and (t3.b > 'ab')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except_second_no_filter_left_join -- +PhysicalResultSink +--hashJoin[LEFT_OUTER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------filter((test_pull_up_predicate_set_op1.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a > 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_left_join -- +PhysicalResultSink +--hashJoin[LEFT_OUTER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a < 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_right_join -- +PhysicalResultSink +--hashJoin[RIGHT_OUTER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_one_side_constant_one_side_column_left_semi_join -- +PhysicalResultSink +--hashJoin[LEFT_SEMI_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a = 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a = 1) and (t3.b > 'ab')) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !except_second_no_filter_right_semi_join -- +PhysicalResultSink +--hashJoin[RIGHT_SEMI_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------filter((test_pull_up_predicate_set_op1.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a > 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a > 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_left_anti_join -- +PhysicalResultSink +--hashJoin[LEFT_ANTI_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----filter((t3.a < 1)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_right_anti_join -- +PhysicalResultSink +--hashJoin[RIGHT_ANTI_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const_full_join -- +PhysicalResultSink +--hashJoin[FULL_OUTER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_all_const_tinyint_int_cross_join -- +PhysicalResultSink +--NestedLoopJoin[CROSS_JOIN] +----PhysicalUnion +----PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_union -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalIntersect +------filter((test_pull_up_predicate_set_op1.a < 1) and (test_pull_up_predicate_set_op1.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----PhysicalUnion +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] + +-- !except_agg -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----PhysicalExcept +------filter((test_pull_up_predicate_set_op1.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] +------filter((test_pull_up_predicate_set_op2.a < 1) and (test_pull_up_predicate_set_op2.b > 'ab')) +--------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------filter((t3.a < 1)) +----------PhysicalOlapScan[test_pull_up_predicate_set_op1] + +-- !union_cross_join -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----NestedLoopJoin[CROSS_JOIN] +------filter((t3.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op3] +------PhysicalOlapScan[test_pull_up_predicate_set_op1] + +-- !union_inner_join -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = t.a) and (t3.b = t.b)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----------filter((test_pull_up_predicate_set_op1.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op1] +----------filter((test_pull_up_predicate_set_op2.a < 1)) +------------PhysicalOlapScan[test_pull_up_predicate_set_op2] +----hashJoin[INNER_JOIN] hashCondition=((t1.a = t2.a)) otherCondition=() +------filter((t3.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op3] +------filter((t2.a < 1)) +--------PhysicalOlapScan[test_pull_up_predicate_set_op1] + +-- !union_all_const_datetime -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_datetimev2 as DATETIMEV2(6)) = expr_cast(b as DATETIMEV2(6))) and (expr_cast(d_smallint as INT) = t.a)) otherCondition=() +----PhysicalUnion +----filter(cast(d_smallint as INT) IN (12222222, 2)) +------PhysicalOlapScan[test_pull_up_predicate_set_op4] + +-- !union_all_const_date -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_datev2 as DATETIMEV2(0)) = expr_cast(b as DATETIMEV2(0))) and (expr_cast(d_smallint as INT) = t.a)) otherCondition=() +----PhysicalUnion +----filter(cast(d_smallint as INT) IN (12222222, 2)) +------PhysicalOlapScan[test_pull_up_predicate_set_op4] + +-- !union_all_const_char100 -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(d_smallint as INT) = t.a) and (t3.d_char100 = t.b)) otherCondition=() +----PhysicalUnion +----filter(cast(d_smallint as INT) IN (12222222, 2) and d_char100 IN ('2024-01-03 10:00:00', '2024-01-03')) +------PhysicalOlapScan[test_pull_up_predicate_set_op4] + +-- !union_all_const_char10 -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.d_char10 = t.b)) otherCondition=() +----PhysicalUnion +----filter(d_char10 IN ('2024-01-03 10:00:00', '2024-01-04')) +------PhysicalOlapScan[test_pull_up_predicate_set_op4] + +-- !union_all_and_project_pull_up -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((t3.a = expr_cast(a as INT)) and (t3.b = t.b)) otherCondition=() +----PhysicalUnion +------PhysicalOlapScan[test_pull_up_predicate_set_op3] +------PhysicalOlapScan[test_pull_up_predicate_set_op3] +----filter((t3.a = 3)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !union_and_const -- +PhysicalResultSink +--hashJoin[INNER_JOIN] hashCondition=((expr_cast(id as INT) = t2.a)) otherCondition=() +----hashAgg[GLOBAL] +------hashAgg[LOCAL] +--------PhysicalUnion +----filter(a IN (1, 2)) +------PhysicalOlapScan[test_pull_up_predicate_set_op3] + +-- !intersect_res -- +0 d2 + +-- !except_res -- +-2 d2 + +-- !union_res -- +-2 d2 +0 d2 + +-- !intersect_one_side_constant_one_side_column_res -- +1 d2 + +-- !intersect_one_side_res -- +0 d2 +1 d2 + +-- !intersect_one_side_no_filter_res -- +0 d2 +1 d2 + +-- !except_first_no_filter_res -- +-2 d2 + +-- !except_second_no_filter_res -- + +-- !except_no_filter_res -- +-2 d2 + +-- !union_different_filter_res -- +-2 d2 +0 d2 +0 d2 +1 d2 + +-- !union_one_side_filter_res -- +-2 d2 +0 d2 +1 d2 + +-- !union_with_const_res -- +-2 d2 +0 d2 +0 d2 + +-- !union_all_const_res -- + +-- !union_all_const_tinyint_int_res -- + +-- !union_all_const_empty_relation_res -- + +-- !union_all_const2_has_cast_to_null_different_type_res -- + +-- !union_all_const2_has_cast_to_null_different_type_res -- + +-- !union_all_different_type_int_cast_to_char_res -- + +-- !union_all_const_char3_char2_res -- + +-- !intersect_one_side_constant_one_side_column_left_join_res -- +1 d2 + +-- !except_second_no_filter_left_join_res -- + +-- !union_left_join_res -- +-3 \N +-2 -2 +0 0 + +-- !union_right_join_res -- +\N 1 +-2 -2 +0 0 + +-- !intersect_one_side_constant_one_side_column_left_semi_join_res -- +1 + +-- !except_second_no_filter_right_semi_join_res -- + +-- !union_left_anti_join_res -- +-3 d2 + +-- !union_right_anti_join_res -- +1 + +-- !union_all_const_full_join_res -- +\N d2 +\N d2 +\N d2 +2 \N +133333 \N + +-- !union_all_const_tinyint_int_cross_join_res -- +2 d2 +2 d2 +2 d2 +3 d2 +3 d2 +3 d2 + +-- !intersect_union_res -- +-3 d2 +-3 d2 +0 d2 +0 d2 + +-- !except_agg_res -- +-2 d2 + +-- !union_cross_join_res -- +-2 d2 +-2 d2 +-2 d2 +-2 d2 +0 d2 +0 d2 +0 d2 +0 d2 + +-- !union_inner_join_res -- +-2 d2 +0 d2 + +-- !union_all_const_datetime_res -- + +-- !union_all_const_date_res -- + +-- !union_all_const_char100_res -- + +-- !union_all_const_char10_res -- +2 2024-01-04 + +-- !union_all_and_project_pull_up_res -- + +-- !union_and_const_res -- +abbbb + +-- !union_child_and_const_exprs_res -- +1 d2 + +-- !union_child_and_const_exprs_andpredicates_res -- + +-- !union_child_and_const_exprs_orpredicates_res -- +1 d2 + diff --git a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out index 06f9705caa1498..25bbc68f24e89b 100644 --- a/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out +++ b/regression-test/data/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.out @@ -51,13 +51,10 @@ PhysicalResultSink -- !except_with_filter -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----PhysicalExcept -------PhysicalDistribute[DistributionSpecHash] +----PhysicalProject +------hashAgg[LOCAL] --------filter((t1.id > 100)) ----------PhysicalOlapScan[t1] -------PhysicalDistribute[DistributionSpecHash] ---------filter((t2.id < 50)) -----------PhysicalOlapScan[t2] -- !intersect_distinct -- PhysicalResultSink @@ -406,15 +403,10 @@ SyntaxError: -- !with_hint_except_with_filter -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----PhysicalExcept -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------filter((t1.id > 100)) -------------PhysicalOlapScan[t1] -------PhysicalDistribute[DistributionSpecHash] ---------hashAgg[LOCAL] -----------filter((t2.id < 50)) -------------PhysicalOlapScan[t2] +----PhysicalProject +------hashAgg[LOCAL] +--------filter((t1.id > 100)) +----------PhysicalOlapScan[t1] Hint log: Used: use_INFER_SET_OPERATOR_DISTINCT @@ -888,13 +880,10 @@ SyntaxError: -- !with_hint_no_except_with_filter -- PhysicalResultSink --PhysicalDistribute[DistributionSpecGather] -----PhysicalExcept -------PhysicalDistribute[DistributionSpecHash] +----PhysicalProject +------hashAgg[LOCAL] --------filter((t1.id > 100)) ----------PhysicalOlapScan[t1] -------PhysicalDistribute[DistributionSpecHash] ---------filter((t2.id < 50)) -----------PhysicalOlapScan[t2] Hint log: Used: diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query8.out index 58865f101ca9c2..00f211d12927a2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query8.out index 58865f101ca9c2..00f211d12927a2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query8.out index 1105499e6b2d03..40a27a12ac0692 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query8.out index 57aa7dc9db3756..5047f4a7878cc1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query8.out index 9270aa516dbf2e..52bfb51f9c2c70 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out index 58865f101ca9c2..00f211d12927a2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query8.out index f46875bd0bcb5b..00f211d12927a2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query8.out index 1105499e6b2d03..40a27a12ac0692 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query8.out index 57aa7dc9db3756..5047f4a7878cc1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query8.out index 9270aa516dbf2e..fd4ec64bb81546 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query8.out index 58865f101ca9c2..a254eeae049f91 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query8.out index 58865f101ca9c2..a254eeae049f91 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query8.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query8.out index 58865f101ca9c2..a254eeae049f91 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query8.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query8.out @@ -29,12 +29,13 @@ PhysicalResultSink ------------------------------PhysicalDistribute[DistributionSpecHash] --------------------------------hashAgg[LOCAL] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject ----------------------------------------filter((customer.c_preferred_cust_flag = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------PhysicalProject --------------------------filter(substring(ca_zip, 1, 5) IN ('10298', '10374', '10425', '11340', '11489', '11618', '11652', '11686', '11855', '11912', '12197', '12318', '12320', '12350', '13086', '13123', '13261', '13338', '13376', '13378', '13443', '13844', '13869', '13918', '14073', '14155', '14196', '14242', '14312', '14440', '14530', '14851', '15371', '15475', '15543', '15734', '15751', '15782', '15794', '16005', '16226', '16364', '16515', '16704', '16791', '16891', '17167', '17193', '17291', '17672', '17819', '17879', '17895', '18218', '18360', '18367', '18410', '18421', '18434', '18569', '18700', '18767', '18829', '18884', '19326', '19444', '19489', '19753', '19833', '19988', '20244', '20317', '20534', '20601', '20712', '21060', '21094', '21204', '21231', '21343', '21727', '21800', '21814', '22728', '22815', '22911', '23065', '23952', '24227', '24255', '24286', '24594', '24660', '24891', '24987', '25115', '25178', '25214', '25264', '25333', '25494', '25717', '25973', '26217', '26689', '27052', '27116', '27156', '27287', '27369', '27385', '27413', '27642', '27700', '28055', '28239', '28571', '28577', '28810', '29086', '29392', '29450', '29752', '29818', '30106', '30415', '30621', '31013', '31016', '31655', '31830', '32489', '32669', '32754', '32919', '32958', '32961', '33113', '33122', '33159', '33467', '33562', '33773', '33869', '34306', '34473', '34594', '34948', '34972', '35076', '35390', '35834', '35863', '35926', '36201', '36335', '36430', '36479', '37119', '37788', '37914', '38353', '38607', '38919', '39214', '39459', '39500', '39503', '40146', '40936', '40979', '41162', '41232', '41255', '41331', '41351', '41352', '41419', '41807', '41836', '41967', '42361', '43432', '43639', '43830', '43933', '44529', '45266', '45484', '45533', '45645', '45676', '45859', '46081', '46131', '46507', '47289', '47369', '47529', '47602', '47770', '48017', '48162', '48333', '48530', '48567', '49101', '49130', '49140', '49211', '49230', '49254', '49472', '50412', '50632', '50636', '50679', '50788', '51089', '51184', '51195', '51634', '51717', '51766', '51782', '51793', '51933', '52094', '52301', '52389', '52868', '53163', '53535', '53565', '54010', '54207', '54364', '54558', '54585', '55233', '55349', '56224', '56355', '56436', '56455', '56600', '56877', '57025', '57553', '57631', '57649', '57839', '58032', '58058', '58062', '58117', '58218', '58412', '58454', '58581', '59004', '59080', '59130', '59226', '59345', '59386', '59494', '59852', '60083', '60298', '60560', '60624', '60736', '61527', '61794', '61860', '61997', '62361', '62585', '62878', '63073', '63180', '63193', '63294', '63792', '63991', '64592', '65148', '65177', '65501', '66057', '66943', '67881', '67975', '67998', '68101', '68293', '68341', '68605', '68730', '68770', '68843', '68852', '68908', '69280', '69952', '69998', '70041', '70070', '70073', '70450', '71144', '71256', '71286', '71836', '71948', '71954', '71997', '72592', '72991', '73021', '73108', '73134', '73146', '73219', '73873', '74686', '75660', '75675', '75742', '75752', '77454', '77817', '78093', '78366', '79077', '79658', '80332', '80846', '81003', '81070', '81084', '81335', '81504', '81755', '81963', '82080', '82602', '82620', '83041', '83086', '83583', '83647', '83833', '83910', '83986', '84247', '84680', '84844', '84919', '85066', '85761', '86057', '86379', '86709', '88086', '88137', '88217', '89193', '89338', '90209', '90229', '90669', '91110', '91894', '92292', '92380', '92645', '92696', '93498', '94791', '94835', '94898', '95042', '95430', '95464', '95694', '96435', '96560', '97173', '97462', '98069', '98072', '98338', '98533', '98569', '98584', '98862', '99060', '99132')) diff --git a/regression-test/regression-test/realData/insert_p0/txn_insert_values_with_schema_change.out b/regression-test/regression-test/realData/insert_p0/txn_insert_values_with_schema_change.out new file mode 100644 index 00000000000000..9e1016fe0e26e7 --- /dev/null +++ b/regression-test/regression-test/realData/insert_p0/txn_insert_values_with_schema_change.out @@ -0,0 +1,13 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !select1 -- +0 a 10 +1 b 20 +2 c 30 +3 d 40 + +-- !select2 -- +0 a 10 +1 b 20 +2 c 30 +3 d 40 + diff --git a/regression-test/suites/nereids_rules_p0/adjust_nullable/set_operation.groovy b/regression-test/suites/nereids_rules_p0/adjust_nullable/set_operation.groovy index a0e2d843dd572d..bbbdf63e03d55b 100644 --- a/regression-test/suites/nereids_rules_p0/adjust_nullable/set_operation.groovy +++ b/regression-test/suites/nereids_rules_p0/adjust_nullable/set_operation.groovy @@ -16,7 +16,9 @@ // under the License. suite("test_set_operation_adjust_nullable") { - + sql "set enable_fallback_to_original_planner=false" + String realDb = context.config.getDbNameByFile(context.file) + logger.info("realDb:${realDb}") sql """ DROP TABLE IF EXISTS set_operation_t1 """ diff --git a/regression-test/suites/nereids_rules_p0/infer_predicate/infer_intersect_except.groovy b/regression-test/suites/nereids_rules_p0/infer_predicate/infer_intersect_except.groovy new file mode 100644 index 00000000000000..fb8ef0a75f98d5 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/infer_predicate/infer_intersect_except.groovy @@ -0,0 +1,208 @@ +// 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("infer_intersect_except") { + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql """SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'""" + sql 'set runtime_filter_mode=off' + sql 'set enable_fold_constant_by_be=true' + sql 'set debug_skip_fold_constant=false' + + + + sql "drop table if exists infer_intersect_except1" + sql "drop table if exists infer_intersect_except2" + sql "drop table if exists infer_intersect_except3" + + sql """ + CREATE TABLE `infer_intersect_except1` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + CREATE TABLE `infer_intersect_except2` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + CREATE TABLE `infer_intersect_except3` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + sql """ + insert into infer_intersect_except1 values(1,'d2',3,5),(0,'d2',3,5),(-3,'d2',2,2),(-2,'d2',2,2); + """ + + sql """ + insert into infer_intersect_except2 values(1,'d2',2,2),(-3,'d2',2,2),(0,'d2',3,5); + """ + + sql """ + insert into infer_intersect_except3 values(1,'d2',2,2),(-2,'d2',2,2),(0,'d2',3,5); + """ + qt_except """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 except select a,b from infer_intersect_except2 where b>'ab' except select a,b from infer_intersect_except2 where a<10; + """ + + qt_except_to_empty """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 except select a,b from infer_intersect_except2 where b>'ab' except select a,b from infer_intersect_except3 where a<0; + """ + + qt_except_not_infer_1_greater_than_0 """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 except select 1,'abc' from infer_intersect_except2 where b>'ab' except select a,b from infer_intersect_except2 where a<0; + """ + + qt_except_number_and_string """ + explain shape plan + select a,2 from infer_intersect_except1 where a>0 except select 1,'abc' from infer_intersect_except2 where b>'ab' except select a,b from infer_intersect_except3 where a<0; + """ + + qt_intersect """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 intersect select a,b from infer_intersect_except2 where b>'ab'; + """ + qt_intersect_empty """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 intersect select a,b from infer_intersect_except2 where a<0; + """ + + qt_intersect_expr """ + explain shape plan + select a+1,b from infer_intersect_except1 where a>0 intersect select a+1,b from infer_intersect_except2 where a+1<0; + """ + + qt_except_and_intersect """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 except select 1,'abc' from infer_intersect_except2 where b>'ab' intersect select a,b from infer_intersect_except3 where a<10; + """ + + qt_except_and_intersect_except_predicate_to_right """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 except select a,'abc' from infer_intersect_except2 where b>'ab' intersect select a,b from infer_intersect_except3 where a<10; + """ + qt_intersect_and_except """ + explain shape plan + select a,b from infer_intersect_except1 where a>0 intersect select 1,'abc' from infer_intersect_except2 where b>'ab' except select a,b from infer_intersect_except3 where a<10; + """ + + qt_function_intersect """ + explain shape plan + select abs(a) from infer_intersect_except1 t1 where abs(a)<3 intersect select abs(a) from infer_intersect_except2 t2 """ + qt_function_except """ + explain shape plan + select abs(a) from infer_intersect_except1 t1 where abs(a)<3 except select abs(a) from infer_intersect_except2 t2 """ + + qt_except_res """ + (select a,b from infer_intersect_except1 where a>0) except (select a,b from infer_intersect_except2 where b>'ab') except (select a,b from infer_intersect_except2 where a<10) order by 1,2; + """ + + qt_except_to_empty_res """ + (select a,b from infer_intersect_except1 where a>0) except (select a,b from infer_intersect_except2 where b>'ab') except (select a,b from infer_intersect_except3 where a<0) order by 1,2; + """ + + qt_except_not_infer_1_greater_than_0_res """ + (select a,b from infer_intersect_except1 where a>0) except (select 1,'abc' from infer_intersect_except2 where b>'ab') except (select a,b from infer_intersect_except2 where a<0) order by 1,2; + """ + + qt_except_number_and_string_res """ + (select a,2 from infer_intersect_except1 where a>0) except (select 1,'abc' from infer_intersect_except2 where b>'ab') except (select a,b from infer_intersect_except3 where a<0) order by 1,2; + """ + + qt_intersect_res """ + (select a,b from infer_intersect_except1 where a>0) intersect (select a,b from infer_intersect_except2 where b>'ab') order by 1,2; + """ + qt_intersect_empty_res """ + (select a,b from infer_intersect_except1 where a>0) intersect (select a,b from infer_intersect_except2 where a<0) order by 1,2; + """ + + qt_intersect_expr_res """ + (select a+1,b from infer_intersect_except1 where a>0) intersect (select a+1,b from infer_intersect_except2 where a+1<0 ) order by 1,2; + """ + + qt_except_and_intersect_res """ + (select a,b from infer_intersect_except1 where a>0) except (select 1,'abc' from infer_intersect_except2 where b>'ab') intersect (select a,b from infer_intersect_except3 where a<10) order by 1,2; + """ + + qt_except_and_intersect_except_predicate_to_right_res """ + (select a,b from infer_intersect_except1 where a>0) except (select a,'abc' from infer_intersect_except2 where b>'ab') intersect (select a,b from infer_intersect_except3 where a<10) order by 1,2; + """ + qt_intersect_and_except_res """ + (select a,b from infer_intersect_except1 where a>0) intersect (select 1,'abc' from infer_intersect_except2 where b>'ab') except (select a,b from infer_intersect_except3 where a<10) order by 1,2; + """ + + qt_function_intersect_res """ + (select abs(a) from infer_intersect_except1 t1 where abs(a)<3) intersect (select abs(a) from infer_intersect_except2 t2) order by 1 """ + qt_function_except_res """ + (select abs(a) from infer_intersect_except1 t1 where abs(a)<3) except (select abs(a) from infer_intersect_except2 t2) order by 1 """ + + sql "drop table if exists infer_intersect_except4" + sql "create table infer_intersect_except4(d_int int, d_char100 char(100), d_smallint smallint, d_tinyint tinyint, d_char10 char(10),d_datetimev2 datetimev2, d_datev2 datev2) properties('replication_num'='1');" + sql """insert into infer_intersect_except4 values(1,'01234567890123456789', 3,3,'0123456789','2020-01-09 10:00:00.99','2020-01-09'),(14,'01234567890123456789', 33,23,'0123456789','2020-01-11 10:00:00.99','2020-01-11') + ,(14,'01234567890123456789', 33,23,'2024-01-04','2020-01-11 10:00:00.99','2020-01-11'), + (14,'01234567890123456789', 33,23,'2024-01-03 10:00:00','2020-01-11 10:00:00.99','2020-01-11');""" + + test { + sql """ + select d_datetimev2 from infer_intersect_except4 where d_datetimev2>'2020-01-01' intersect select d_int from infer_intersect_except4 where d_int<10; + """ + exception("can not cast from origin type DATETIMEV2(0) to target type=UNSUPPORTED") + } + qt_different_type_date_string """ + explain shape plan + select d_datetimev2 from infer_intersect_except4 where d_datetimev2>'2020-01-01' intersect select d_char100 from infer_intersect_except4 where d_char100<'abc'; + """ + qt_different_type_int_string """ + explain shape plan + select d_int from infer_intersect_except4 where d_int>2 intersect select d_char100 from infer_intersect_except4 where d_char100<'abc'; + """ + qt_different_type_date_string_res """ + (select d_datetimev2 from infer_intersect_except4 where d_datetimev2>'2020-01-01') intersect (select d_char100 from infer_intersect_except4 where d_char100<'abc') order by 1; + """ + qt_different_type_int_string_res """ + (select d_int from infer_intersect_except4 where d_int>2 ) intersect (select d_char100 from infer_intersect_except4 where d_char100<'abc') order by 1; + """ +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy index 1eb5540c812193..06998e0e438be1 100644 --- a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy +++ b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_literal.groovy @@ -20,6 +20,9 @@ suite("test_pull_up_predicate_literal") { sql "set enable_fallback_to_original_planner=false" sql """SET ignore_shape_nodes='PhysicalDistribute'""" sql 'set runtime_filter_mode=off' + sql 'set enable_fold_constant_by_be=true' + sql 'set debug_skip_fold_constant=false' + sql 'set disable_join_reorder=true' sql """ CREATE TABLE `test_pull_up_predicate_literal` ( @@ -50,9 +53,11 @@ suite("test_pull_up_predicate_literal") { ) tmp inner join test_pull_up_predicate_literal ds on tmp.col1 = ds.col1 and tmp.col2 = ds.col2; """ + qt_test_pull_up_literal1 """ + explain shape plan select * from test_pull_up_predicate_literal_view; + """ - - qt_test_pull_up_literal """explain shape plan select * from test_pull_up_predicate_literal_view where col1='abc' and col2='def';""" + qt_test_pull_up_literal2 """explain shape plan select * from test_pull_up_predicate_literal_view where col1='abc' and col2='def';""" qt_test_pull_up_literal_suquery """ explain shape plan SELECT * diff --git a/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy new file mode 100644 index 00000000000000..4afe59e2d316c4 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/infer_predicate/pull_up_predicate_set_op.groovy @@ -0,0 +1,469 @@ +// 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_pull_up_predicate_set_op") { + sql "set enable_nereids_planner=true" + sql "set enable_fallback_to_original_planner=false" + sql """SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'""" + sql 'set runtime_filter_mode=off' + sql 'set enable_fold_constant_by_be=true' + sql 'set debug_skip_fold_constant=false' + sql 'set disable_join_reorder=true' + + sql "drop table if exists test_pull_up_predicate_set_op1" + sql "drop table if exists test_pull_up_predicate_set_op2" + sql "drop table if exists test_pull_up_predicate_set_op3" + + sql """ + CREATE TABLE `test_pull_up_predicate_set_op1` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + CREATE TABLE `test_pull_up_predicate_set_op2` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + CREATE TABLE `test_pull_up_predicate_set_op3` ( + `a` INT NULL, + `b` VARCHAR(10) NULL, + `c` INT NULL, + `d` INT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`a`, `b`) + DISTRIBUTED BY RANDOM BUCKETS AUTO + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + """ + + sql """ + insert into test_pull_up_predicate_set_op1 values(1,'d2',3,5),(0,'d2',3,5),(-3,'d2',2,2),(-2,'d2',2,2); + """ + + sql """ + insert into test_pull_up_predicate_set_op2 values(1,'d2',2,2),(-3,'d2',2,2),(0,'d2',3,5); + """ + + sql """ + insert into test_pull_up_predicate_set_op3 values(1,'d2',2,2),(-2,'d2',2,2),(0,'d2',3,5); + """ + + sql "drop table if exists test_pull_up_predicate_set_op4" + sql "create table test_pull_up_predicate_set_op4(d_int int, d_char100 char(100), d_smallint smallint, d_tinyint tinyint, d_char10 char(10),d_datetimev2 datetimev2, d_datev2 datev2) properties('replication_num'='1');" + sql """insert into test_pull_up_predicate_set_op4 values(1,'01234567890123456789', 3,3,'0123456789','2020-01-09 10:00:00.99','2020-01-09'),(14,'01234567890123456789', 33,23,'0123456789','2020-01-11 10:00:00.99','2020-01-11') + ,(14,'01234567890123456789', 33,23,'2024-01-04','2020-01-11 10:00:00.99','2020-01-11'), + (14,'01234567890123456789', 33,23,'2024-01-03 10:00:00','2020-01-11 10:00:00.99','2020-01-11');""" + + qt_intersect """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_intersect_one_side_constant_one_side_column """ + explain shape plan + select t.a,t3.b from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_intersect_one_side """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_intersect_one_side_no_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_except_first_no_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except_second_no_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except_no_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 except select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_different_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<2 union all select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_one_side_filter """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_with_const """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union all select a,b from test_pull_up_predicate_set_op2 where a<1 union all select 2,2) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_all_const """explain shape plan + select t.a,t3.b from (select 133333 as a,'aa' as b union all select 2,'dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + qt_union_all_const_tinyint_int """explain shape plan + select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + + qt_union_all_const_empty_relation """ explain shape plan select t.a,t3.b from (select 3 as a,'aa' as b from test_pull_up_predicate_set_op3 limit 0 offset 0 union all select 2,'dd' ) t + inner join test_pull_up_predicate_set_op3 t3 on t3.a=t.a and t3.b=t.b;""" + + qt_union_all_const2_has_cast_to_null_different_type """ explain shape plan + select t.a,t3.b from (select 3 as a,'aa' as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; """ + + qt_union_all_const2_has_cast_to_null_different_type """ explain shape plan + select t.a,t3.b from (select 'abcd' as a,12 as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; """ + qt_union_all_different_type_int_cast_to_char """ explain shape plan + select t.a,t3.b from (select 'abcd' as a,12 as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + + qt_union_all_const_char3_char2 """explain shape plan + select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' union all select 5,'aab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_child_and_const_exprs """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) union select a,b from test_pull_up_predicate_set_op2 where a in (1,2) union select 2,2 union select 2,2) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_child_and_const_exprs_andpredicates """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) and b in ('2d','3') union select 2,'2d' union select 2,'3') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_child_and_const_exprs_orpredicates """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) or b in ('2d','3') union select 2,'2d' union select 2,'3') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + + // test_different join type + qt_intersect_one_side_constant_one_side_column_left_join """ + explain shape plan + select t.a,t3.b from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except_second_no_filter_left_join """ + explain shape plan + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t left join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_left_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t left join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_right_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t right join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_intersect_one_side_constant_one_side_column_left_semi_join """ + explain shape plan + select t.a from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t left semi join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except_second_no_filter_right_semi_join """ + explain shape plan + select t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t right semi join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_left_anti_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t left anti join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_right_anti_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t right anti join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_all_const_full_join """explain shape plan + select t.a,t3.b from (select 133333 as a,'aa' as b union all select 2,'dd' ) t full outer join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + qt_union_all_const_tinyint_int_cross_join """explain shape plan + select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' ) t cross join test_pull_up_predicate_set_op3 t3 + """ + + // join right is join or union + qt_intersect_union """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join + (select a,b from test_pull_up_predicate_set_op2 union all select a,b from test_pull_up_predicate_set_op2 ) t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_except_agg """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join + (select a,b from test_pull_up_predicate_set_op1 group by a,b) t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_cross_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join + (select t1.a,t2.b from test_pull_up_predicate_set_op3 t1 cross join test_pull_up_predicate_set_op1 t2 ) t3 + on t3.a=t.a and t3.b=t.b; + """ + qt_union_inner_join """ + explain shape plan + select * from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join + (select t1.a,t2.b from test_pull_up_predicate_set_op3 t1 inner join test_pull_up_predicate_set_op1 t2 on t1.a=t2.a ) t3 + on t3.a=t.a and t3.b=t.b; + """ + + qt_union_all_const_datetime """ + explain shape plan + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_datetimev2=t.b; + """ + qt_union_all_const_date """ + explain shape plan + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_datev2=t.b; + """ + qt_union_all_const_char100 """ + explain shape plan + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_char100=t.b; + """ + qt_union_all_const_char10 """ explain shape plan + select t.a,t3.d_char10 from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-04') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_char10=t.b;""" + + qt_union_all_and_project_pull_up """explain shape plan + select t.a,t3.b from (select 3 as a,b from test_pull_up_predicate_set_op3 union all select 3,b from test_pull_up_predicate_set_op3 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b; + """ + // need pull up from agg support + qt_union_and_const """explain shape plan + select c2 from (select 2 id,'abc' c2 union select 1 ,'abbbb' c4 ) t inner join test_pull_up_predicate_set_op3 t2 on t.id=t2.a""" + + // test sql res + qt_intersect_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_except_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; """ + + qt_intersect_one_side_constant_one_side_column_res """ + select t.a,t3.b from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_intersect_one_side_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_intersect_one_side_no_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_except_first_no_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_except_second_no_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_except_no_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 except select a,b from test_pull_up_predicate_set_op2 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_union_different_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<2 union all select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_union_one_side_filter_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_union_with_const_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union all select a,b from test_pull_up_predicate_set_op2 where a<1 union all select 2,2) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_union_all_const_res """select t.a,t3.b from (select 133333 as a,'aa' as b union all select 2,'dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b;""" + qt_union_all_const_tinyint_int_res """select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2;""" + + qt_union_all_const_empty_relation_res """ select t.a,t3.b from (select 3 as a,'aa' as b from test_pull_up_predicate_set_op3 limit 0 offset 0 union all select 2,'dd' ) t + inner join test_pull_up_predicate_set_op3 t3 on t3.a=t.a and t3.b=t.b order by 1,2;""" + + qt_union_all_const2_has_cast_to_null_different_type_res """ select t.a,t3.b from (select 3 as a,'aa' as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; """ + + qt_union_all_const2_has_cast_to_null_different_type_res """ select t.a,t3.b from (select 'abcd' as a,12 as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; """ + qt_union_all_different_type_int_cast_to_char_res """ select t.a,t3.b from (select 'abcd' as a,12 as b union all select 'abc','dd' ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2;""" + + qt_union_all_const_char3_char2_res """select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' union all select 5,'aab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_intersect_one_side_constant_one_side_column_left_join_res """ + select t.a,t3.b from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_except_second_no_filter_left_join_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t left join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_left_join_res """ + select t.a,t3.a from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t left join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_right_join_res """ + select t.a,t3.a from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t right join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_intersect_one_side_constant_one_side_column_left_semi_join_res """ + select t.a from (select 1 as a,b from test_pull_up_predicate_set_op1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t left semi join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1; + """ + qt_except_second_no_filter_right_semi_join_res """ + select t3.b from (select a,b from test_pull_up_predicate_set_op1 where a>1 except select a,b from test_pull_up_predicate_set_op2 ) t right semi join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1; + """ + qt_union_left_anti_join_res """ + select t.a,t.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t left anti join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_right_anti_join_res """ + select t3.a from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t right anti join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1; + """ + qt_union_all_const_full_join_res """select t.a,t3.b from (select 133333 as a,'aa' as b union all select 2,'dd' ) t full outer join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2;""" + qt_union_all_const_tinyint_int_cross_join_res """select t.a,t3.b from (select 3 as a,'aa' as b union all select 2,'dd' ) t cross join test_pull_up_predicate_set_op3 t3 order by 1,2 + """ + + qt_intersect_union_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 intersect select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join + (select a,b from test_pull_up_predicate_set_op2 union all select a,b from test_pull_up_predicate_set_op2 ) t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_except_agg_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 except select a,b from test_pull_up_predicate_set_op2 where b>'ab') t inner join + (select a,b from test_pull_up_predicate_set_op1 group by a,b) t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_cross_join_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join + (select t1.a,t2.b from test_pull_up_predicate_set_op3 t1 cross join test_pull_up_predicate_set_op1 t2 ) t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_inner_join_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a<1 union select a,b from test_pull_up_predicate_set_op2 where a<1) t inner join + (select t1.a,t2.b from test_pull_up_predicate_set_op3 t1 inner join test_pull_up_predicate_set_op1 t2 on t1.a=t2.a ) t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + + qt_union_all_const_datetime_res """ + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_datetimev2=t.b order by 1; + """ + qt_union_all_const_date_res """ + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_datev2=t.b order by 1; + """ + qt_union_all_const_char100_res """ + select t.a from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-03') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_smallint=t.a and t3.d_char100=t.b order by 1; + """ + qt_union_all_const_char10_res """ select t.a,t3.d_char10 from (select 12222222 as a,'2024-01-03 10:00:00' as b union all select 2,'2024-01-04') t inner join test_pull_up_predicate_set_op4 t3 + on t3.d_char10=t.b order by 1,2;""" + + qt_union_all_and_project_pull_up_res """select t.a,t3.b from (select 3 as a,b from test_pull_up_predicate_set_op3 union all select 3,b from test_pull_up_predicate_set_op3 ) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + // need pull up from agg support + qt_union_and_const_res """select c2 from (select 2 id,'abc' c2 union select 1 ,'abbbb' c4 ) t inner join test_pull_up_predicate_set_op3 t2 on t.id=t2.a order by 1""" + + qt_union_child_and_const_exprs_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) union select a,b from test_pull_up_predicate_set_op2 where a in (1,2) union select 2,2 union select 2,2) t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_child_and_const_exprs_andpredicates_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) and b in ('2d','3') union select 2,'2d' union select 2,'3') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2; + """ + qt_union_child_and_const_exprs_orpredicates_res """ + select t.a,t3.b from (select a,b from test_pull_up_predicate_set_op1 where a in (1,2) or b in ('2d','3') union select 2,'2d' union select 2,'3') t inner join test_pull_up_predicate_set_op3 t3 + on t3.a=t.a and t3.b=t.b order by 1,2;""" +} \ No newline at end of file diff --git a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy index 9c33f5a9f45312..d29bac550fa6e3 100644 --- a/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy +++ b/regression-test/suites/nereids_rules_p0/infer_set_operator_distinct/infer_set_operator_distinct.groovy @@ -22,7 +22,6 @@ suite("infer_set_operator_distinct") { sql "set disable_nereids_rules=PRUNE_EMPTY_PARTITION" sql "set enable_parallel_result_sink=false;" - sql """ DROP TABLE IF EXISTS t1; """