From aec23f112f5077d2853d80ccc1fdbbc0725fa3ad Mon Sep 17 00:00:00 2001 From: seawinde Date: Mon, 18 Dec 2023 21:04:13 +0800 Subject: [PATCH] query rewrite support filter valid partition filter --- .../java/org/apache/doris/catalog/MTMV.java | 6 +- .../doris/mtmv/MTMVRelationManager.java | 2 +- .../apache/doris/nereids/rules/RuleSet.java | 8 + ...AbstractMaterializedViewAggregateRule.java | 15 + .../mv/AbstractMaterializedViewJoinRule.java | 8 + .../mv/AbstractMaterializedViewRule.java | 41 +- .../mv/MaterializationContext.java | 2 +- .../mv/MaterializedViewFilterJoinRule.java | 47 ++ ...MaterializedViewFilterProjectJoinRule.java | 49 ++ .../mv/MaterializedViewOnlyJoinRule.java | 46 ++ ...MaterializedViewProjectFilterJoinRule.java | 49 ++ .../mv/MaterializedViewProjectJoinRule.java | 2 +- .../expressions/functions/agg/Count.java | 1 - .../commands/UpdateMvByPartitionCommand.java | 2 +- .../doris/nereids/util/ExpressionUtils.java | 7 +- .../mv/aggregate_with_roll_up.out | 128 +++- .../mv/aggregate_without_roll_up.out | 127 +++- .../data/nereids_rules_p0/mv/inner_join.out | 230 +++++- .../data/nereids_rules_p0/mv/outer_join.out | 178 ++++- .../mv/aggregate_with_roll_up.groovy | 693 +++++++++++++++--- .../mv/aggregate_without_roll_up.groovy | 619 ++++++++++++++-- .../nereids_rules_p0/mv/inner_join.groovy | 311 ++++++-- .../nereids_rules_p0/mv/outer_join.groovy | 248 ++++--- .../mv/partition_mv_rewrite.groovy | 202 +++++ 24 files changed, 2656 insertions(+), 365 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java create mode 100644 regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index efe426bac22140..71be5462938e34 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -35,6 +35,7 @@ import org.apache.doris.mtmv.MTMVRelation; import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.persist.gson.GsonUtils; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; @@ -189,12 +190,13 @@ public Set getExcludedTriggerTables() { return Sets.newHashSet(split); } - public MTMVCache getOrGenerateCache() throws AnalysisException { + // this should use the same connectContext with query, to use the same session variable + public MTMVCache getOrGenerateCache(ConnectContext parent) throws AnalysisException { if (cache == null) { writeMvLock(); try { if (cache == null) { - this.cache = MTMVCache.from(this, MTMVPlanUtil.createMTMVContext(this)); + this.cache = MTMVCache.from(this, parent); } } finally { writeMvUnlock(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java index e83b109d3cef49..b418d29e8d3104 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java @@ -51,7 +51,7 @@ public class MTMVRelationManager implements MTMVHookService { public Set getMtmvsByBaseTable(BaseTableInfo table) { Set baseTableInfos = tableMTMVs.get(table); - return baseTableInfos == null? ImmutableSet.of() : baseTableInfos; + return baseTableInfos == null ? ImmutableSet.of() : baseTableInfos; } public Set getAvailableMTMVs(List tableInfos) { 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 a937472c862fbc..3e67f038967a68 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 @@ -41,7 +41,11 @@ import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTranspose; import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTransposeProject; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewAggregateRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterJoinRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterProjectJoinRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewOnlyJoinRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectAggregateRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterJoinRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule; import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; @@ -224,7 +228,11 @@ public class RuleSet { .build(); public static final List MATERIALIZED_VIEW_RULES = planRuleFactories() + .add(MaterializedViewOnlyJoinRule.INSTANCE) .add(MaterializedViewProjectJoinRule.INSTANCE) + .add(MaterializedViewFilterJoinRule.INSTANCE) + .add(MaterializedViewFilterProjectJoinRule.INSTANCE) + .add(MaterializedViewProjectFilterJoinRule.INSTANCE) .add(MaterializedViewAggregateRule.INSTANCE) .add(MaterializedViewProjectAggregateRule.INSTANCE) .build(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index 9b6e1197ef14d3..5b56c0afe6fdd4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -47,6 +47,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.ArrayList; import java.util.HashMap; @@ -64,6 +66,8 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate protected static final Map AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP = new HashMap<>(); + protected final String currentClassName = this.getClass().getSimpleName(); + private final Logger logger = LogManager.getLogger(this.getClass()); static { AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.put( @@ -81,10 +85,12 @@ protected Plan rewriteQueryByView(MatchMode matchMode, // get view and query aggregate and top plan correspondingly Pair> viewTopPlanAndAggPair = splitToTopPlanAndAggregate(viewStructInfo); if (viewTopPlanAndAggPair == null) { + logger.info(currentClassName + "split to view to top plan and agg fail so return null"); return null; } Pair> queryTopPlanAndAggPair = splitToTopPlanAndAggregate(queryStructInfo); if (queryTopPlanAndAggPair == null) { + logger.info(currentClassName + "split to query to top plan and agg fail so return null"); return null; } // Firstly, handle query group by expression rewrite @@ -113,6 +119,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, true); if (rewrittenQueryGroupExpr.isEmpty()) { // can not rewrite, bail out. + logger.info(currentClassName + " can not rewrite expression when not need roll up"); return null; } return new LogicalProject<>( @@ -127,12 +134,14 @@ protected Plan rewriteQueryByView(MatchMode matchMode, viewExpr -> viewExpr.anyMatch(expr -> expr instanceof AggregateFunction && ((AggregateFunction) expr).isDistinct()))) { // if mv aggregate function contains distinct, can not roll up, bail out. + logger.info(currentClassName + " view contains distinct function so can not roll up"); return null; } // split the query top plan expressions to group expressions and functions, if can not, bail out. Pair, Set> queryGroupAndFunctionPair = topPlanSplitToGroupAndFunction(queryTopPlanAndAggPair); if (queryGroupAndFunctionPair == null) { + logger.info(currentClassName + " query top plan split to group by and function fail so return null"); return null; } // Secondly, try to roll up the agg functions @@ -159,6 +168,8 @@ protected Plan rewriteQueryByView(MatchMode matchMode, Function rollupAggregateFunction = rollup(queryFunction, queryFunctionShuttled, mvExprToMvScanExprQueryBased); if (rollupAggregateFunction == null) { + logger.info(currentClassName + " query function " + queryFunction.getName() + + " can not roll up so return null"); return null; } // key is query need roll up expr, value is mv scan based roll up expr @@ -170,6 +181,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryToViewSlotMapping, false); if (rewrittenFunctionExpression == null) { + logger.info(currentClassName + " roll up expression can not rewrite by view so return null"); return null; } finalAggregateExpressions.add((NamedExpression) rewrittenFunctionExpression); @@ -179,6 +191,8 @@ protected Plan rewriteQueryByView(MatchMode matchMode, ExpressionUtils.shuttleExpressionWithLineage(topExpression, queryTopPlan); if (!mvExprToMvScanExprQueryBased.containsKey(queryGroupShuttledExpr)) { // group expr can not rewrite by view + logger.info(currentClassName + + " view group expressions can not contains the query group by expression so return null"); return null; } groupRewrittenExprMap.put(queryGroupShuttledExpr, @@ -191,6 +205,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryToViewSlotMapping, true); if (rewrittenGroupExpression == null) { + logger.info(currentClassName + " query top expression can not be rewritten by view so return null"); return null; } finalAggregateExpressions.add((NamedExpression) rewrittenGroupExpression); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java index aad1ffa529c5a1..19b05b71e19f85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java @@ -27,6 +27,9 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import java.util.List; import java.util.stream.Collectors; @@ -35,6 +38,10 @@ * This is responsible for common join rewriting */ public abstract class AbstractMaterializedViewJoinRule extends AbstractMaterializedViewRule { + + protected final String currentClassName = this.getClass().getSimpleName(); + private final Logger logger = LogManager.getLogger(this.getClass()); + @Override protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, @@ -53,6 +60,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, // Can not rewrite, bail out if (expressionsRewritten.isEmpty() || expressionsRewritten.stream().anyMatch(expr -> !(expr instanceof NamedExpression))) { + logger.info(currentClassName + " expression to rewrite is not named expr so return null"); return null; } // record the group id in materializationContext, and when rewrite again in diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index 0bedf4545810d6..c4ea2456dad95c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -30,6 +30,7 @@ import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; import org.apache.doris.nereids.rules.exploration.mv.mapping.EquivalenceClassSetMapping; import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping; @@ -71,6 +72,7 @@ public abstract class AbstractMaterializedViewRule { public static final HashSet SUPPORTED_JOIN_TYPE_SET = Sets.newHashSet(JoinType.INNER_JOIN, JoinType.LEFT_OUTER_JOIN); + protected final String currentClassName = this.getClass().getSimpleName(); private final Logger logger = LogManager.getLogger(this.getClass()); /** @@ -81,6 +83,7 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { List materializationContexts = cascadesContext.getMaterializationContexts(); List rewriteResults = new ArrayList<>(); if (materializationContexts.isEmpty()) { + logger.info(currentClassName + " materializationContexts is empty so return"); return rewriteResults; } @@ -88,6 +91,7 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { // TODO Just Check query queryPlan firstly, support multi later. StructInfo queryStructInfo = queryStructInfos.get(0); if (!checkPattern(queryStructInfo)) { + logger.info(currentClassName + " queryStructInfo is not valid so return"); return rewriteResults; } @@ -96,44 +100,51 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { if (queryPlan.getGroupExpression().isPresent() && materializationContext.alreadyRewrite( queryPlan.getGroupExpression().get().getOwnerGroup().getGroupId())) { + logger.info(currentClassName + " this group is already rewritten so skip"); continue; } MTMV mtmv = materializationContext.getMtmv(); - MTMVCache mtmvCache = getCacheFromMTMV(mtmv); + MTMVCache mtmvCache = getCacheFromMTMV(mtmv, cascadesContext); if (mtmvCache == null) { - return null; + logger.info(currentClassName + " mv cache is null so return"); + return rewriteResults; } List viewStructInfos = extractStructInfo(mtmvCache.getLogicalPlan(), cascadesContext); if (viewStructInfos.size() > 1) { // view struct info should only have one + logger.info(currentClassName + " the num of view struct info is more then one so return"); return rewriteResults; } StructInfo viewStructInfo = viewStructInfos.get(0); if (!checkPattern(viewStructInfo)) { + logger.info(currentClassName + " viewStructInfo is not valid so return"); continue; } MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations()); if (MatchMode.COMPLETE != matchMode) { + logger.info(currentClassName + " match mode is not complete so return"); continue; } List queryToViewTableMappings = RelationMapping.generate(queryStructInfo.getRelations(), viewStructInfo.getRelations()); // if any relation in query and view can not map, bail out. if (queryToViewTableMappings == null) { + logger.info(currentClassName + " query to view table mapping null so return"); return rewriteResults; } for (RelationMapping queryToViewTableMapping : queryToViewTableMappings) { SlotMapping queryToViewSlotMapping = SlotMapping.generate(queryToViewTableMapping); if (queryToViewSlotMapping == null) { + logger.info(currentClassName + " query to view slot mapping null so continue"); continue; } LogicalCompatibilityContext compatibilityContext = LogicalCompatibilityContext.from(queryToViewTableMapping, queryToViewSlotMapping, queryStructInfo, viewStructInfo); - // todo outer join compatibility check List pulledUpExpressions = StructInfo.isGraphLogicalEquals(queryStructInfo, viewStructInfo, compatibilityContext); if (pulledUpExpressions == null) { + logger.info(currentClassName + " graph logical is not equals so continue"); continue; } // set pulled up expression to queryStructInfo predicates and update related predicates @@ -144,6 +155,7 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { queryToViewSlotMapping); // Can not compensate, bail out if (compensatePredicates.isEmpty()) { + logger.info(currentClassName + " predicate compensate fail so continue"); continue; } Plan rewritedPlan; @@ -159,6 +171,7 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { queryToViewSlotMapping, true); if (rewriteCompensatePredicates.isEmpty()) { + logger.info(currentClassName + " compensate predicate rewrite by view fail so continue"); continue; } rewritedPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan); @@ -171,11 +184,20 @@ protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { rewritedPlan, materializationContext); if (rewritedPlan == null) { + logger.info(currentClassName + " rewrite query by view fail so continue"); continue; } if (!checkPartitionIsValid(queryStructInfo, materializationContext, cascadesContext)) { + logger.info(currentClassName + " check partition validation fail so continue"); continue; } + // run rbo job on mv rewritten plan + CascadesContext rewrittenPlanContext = + CascadesContext.initContext(cascadesContext.getStatementContext(), rewritedPlan, + cascadesContext.getCurrentJobContext().getRequiredProperties()); + Rewriter.getWholeTreeRewriter(cascadesContext).execute(); + rewritedPlan = rewrittenPlanContext.getRewritePlan(); + logger.info(currentClassName + "rewrite by materialized view success"); rewriteResults.add(rewritedPlan); } } @@ -246,12 +268,12 @@ protected boolean checkPartitionIsValid( return relatedTalbeValidSet.containsAll(relatedTableSelectedPartitionToCheck); } - private MTMVCache getCacheFromMTMV(MTMV mtmv) { + private MTMVCache getCacheFromMTMV(MTMV mtmv, CascadesContext cascadesContext) { MTMVCache cache; try { - cache = mtmv.getOrGenerateCache(); + cache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext()); } catch (AnalysisException analysisException) { - logger.warn("get mtmv cache analysisException", analysisException); + logger.warn(this.getClass().getSimpleName() + " get mtmv cache analysisException", analysisException); return null; } return cache; @@ -319,6 +341,7 @@ protected List rewriteExpression( targetToTargetReplacementMapping); if (replacedExpression.anyMatch(slotsToRewrite::contains)) { // if contains any slot to rewrite, which means can not be rewritten by target, bail out + logger.info(currentClassName + "any slot can not rewrite by view so rewrite expression fail"); return ImmutableList.of(); } Expression sourceExpression = sourceExpressionsToWrite.get(index); @@ -366,6 +389,7 @@ protected SplitPredicate predicatesCompensate( .toSlotReferenceMap(); EquivalenceClass viewEquivalenceClassQueryBased = viewEquivalenceClass.permute(viewToQuerySlotMapping); if (viewEquivalenceClassQueryBased == null) { + logger.info(currentClassName + " permute view equivalence class by query fail so return empty"); return SplitPredicate.empty(); } final List equalCompensateConjunctions = new ArrayList<>(); @@ -374,6 +398,7 @@ protected SplitPredicate predicatesCompensate( } if (queryEquivalenceClass.isEmpty() && !viewEquivalenceClass.isEmpty()) { + logger.info(currentClassName + " view has equivalence class but query not so return empty"); return SplitPredicate.empty(); } EquivalenceClassSetMapping queryToViewEquivalenceMapping = @@ -381,6 +406,7 @@ protected SplitPredicate predicatesCompensate( // can not map all target equivalence class, can not compensate if (queryToViewEquivalenceMapping.getEquivalenceClassSetMap().size() < viewEquivalenceClass.getEquivalenceSetList().size()) { + logger.info(currentClassName + " view has more equivalence than query so return empty"); return SplitPredicate.empty(); } // do equal compensate @@ -428,6 +454,7 @@ protected SplitPredicate predicatesCompensate( // query range predicate can not contain all view range predicate when view have range predicate, bail out if (!viewRangePredicateQueryBased.equals(BooleanLiteral.TRUE) && !queryRangeSet.containsAll(viewRangeQueryBasedSet)) { + logger.info(currentClassName + " query range predicate set can not contains all view range predicate"); return SplitPredicate.empty(); } queryRangeSet.removeAll(viewRangeQueryBasedSet); @@ -447,6 +474,8 @@ protected SplitPredicate predicatesCompensate( // bail out if (!viewResidualPredicateQueryBased.equals(BooleanLiteral.TRUE) && !queryResidualSet.containsAll(viewResidualQueryBasedSet)) { + logger.info( + currentClassName + " query residual predicate set can not contains all view residual predicate"); return SplitPredicate.empty(); } queryResidualSet.removeAll(viewResidualQueryBasedSet); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java index ab7e540f186068..dedc775a3ebf95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java @@ -67,7 +67,7 @@ public MaterializationContext(MTMV mtmv, MTMVCache mtmvCache = null; try { - mtmvCache = mtmv.getOrGenerateCache(); + mtmvCache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext()); } catch (AnalysisException e) { LOG.warn("MaterializationContext init mv cache generate fail", e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java new file mode 100644 index 00000000000000..406afdc48fe77b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java @@ -0,0 +1,47 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RulePromise; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This is responsible for join pattern such as filter on join + */ +public class MaterializedViewFilterJoinRule extends AbstractMaterializedViewJoinRule implements RewriteRuleFactory { + + public static final MaterializedViewFilterJoinRule INSTANCE = new MaterializedViewFilterJoinRule(); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalFilter(logicalJoin(any(), any())).thenApplyMulti(ctx -> { + LogicalFilter> root = ctx.root; + return rewrite(root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_JOIN, RulePromise.EXPLORE)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java new file mode 100644 index 00000000000000..c7e869e04db61c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RulePromise; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This is responsible for join pattern such as filter on project on join + */ +public class MaterializedViewFilterProjectJoinRule extends AbstractMaterializedViewJoinRule + implements RewriteRuleFactory { + + public static final MaterializedViewFilterProjectJoinRule INSTANCE = new MaterializedViewFilterProjectJoinRule(); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalFilter(logicalProject(logicalJoin(any(), any()))).thenApplyMulti(ctx -> { + LogicalFilter>> root = ctx.root; + return rewrite(root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_JOIN, RulePromise.EXPLORE)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java new file mode 100644 index 00000000000000..16a66e959e485c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java @@ -0,0 +1,46 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RulePromise; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This is responsible for join pattern such as only join + */ +public class MaterializedViewOnlyJoinRule extends AbstractMaterializedViewJoinRule implements RewriteRuleFactory { + + public static final MaterializedViewOnlyJoinRule INSTANCE = new MaterializedViewOnlyJoinRule(); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalJoin(any(), any()).thenApplyMulti(ctx -> { + LogicalJoin root = ctx.root; + return rewrite(root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_JOIN, RulePromise.EXPLORE)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java new file mode 100644 index 00000000000000..e624b7f6a008f7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.java @@ -0,0 +1,49 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RulePromise; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This is responsible for join pattern such as project on filter on join + */ +public class MaterializedViewProjectFilterJoinRule extends AbstractMaterializedViewJoinRule + implements RewriteRuleFactory { + + public static final MaterializedViewProjectFilterJoinRule INSTANCE = new MaterializedViewProjectFilterJoinRule(); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalProject(logicalFilter(logicalJoin(any(), any()))).thenApplyMulti(ctx -> { + LogicalProject>> root = ctx.root; + return rewrite(root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_JOIN, RulePromise.EXPLORE)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java index 456d1ce24a0519..3a81b9ae626236 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java @@ -30,7 +30,7 @@ import java.util.List; /** - * This is responsible for join rewriting according to different pattern + * This is responsible for join pattern such as project on join * */ public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoinRule implements RewriteRuleFactory { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java index 6b34b763c73cf4..e1fc003249a9be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java @@ -23,7 +23,6 @@ import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; import org.apache.doris.nereids.trees.expressions.functions.Function; -import org.apache.doris.nereids.trees.expressions.functions.scalar.BitmapCount; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index 9d85a7357f02e9..ae4e048f1c6263 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -113,7 +113,7 @@ private static Set constructPredicates(Set partitions .collect(ImmutableSet.toImmutableSet()); } - public static Expression convertPartitionItemToPredicate(PartitionItem item, Slot col) { + private static Expression convertPartitionItemToPredicate(PartitionItem item, Slot col) { if (item instanceof ListPartitionItem) { List inValues = ((ListPartitionItem) item).getItems().stream() .map(key -> Literal.fromLegacyLiteral(key.getKeys().get(0), 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 f080eace3971a6..70b91dfe1028c1 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 @@ -362,12 +362,7 @@ private ExpressionReplacer() { @Override public Expression visit(Expression expr, Map replaceMap) { if (replaceMap.containsKey(expr)) { - Expression replacedExpression = replaceMap.get(expr); - if (replacedExpression instanceof SlotReference - && replacedExpression.nullable() != expr.nullable()) { - replacedExpression = ((SlotReference) replacedExpression).withNullable(expr.nullable()); - } - return replacedExpression; + return replaceMap.get(expr); } return super.visit(expr, replaceMap); } diff --git a/regression-test/data/nereids_rules_p0/mv/aggregate_with_roll_up.out b/regression-test/data/nereids_rules_p0/mv/aggregate_with_roll_up.out index 89514050c1092d..b240c02f7eaaf2 100644 --- a/regression-test/data/nereids_rules_p0/mv/aggregate_with_roll_up.out +++ b/regression-test/data/nereids_rules_p0/mv/aggregate_with_roll_up.out @@ -1,15 +1,123 @@ -- This file is automatically generated. You should know what you did if you want to edit this --- !query1_0_before -- -4 208.70 -6 109.20 +-- !query13_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 --- !query1_0_after -- -4 208.70 -6 109.20 +-- !query13_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 --- !query2_0_before -- -2 3 2023-12-11 109.20 +-- !query14_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 \N \N \N 1 0 +2 4 2023-12-10 \N \N \N 1 0 +3 3 2023-12-11 \N \N \N 1 0 +4 3 2023-12-09 \N \N \N 1 0 --- !query2_0_after -- -2 3 2023-12-11 109.20 +-- !query14_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 \N \N \N 1 0 +2 4 2023-12-10 \N \N \N 1 0 +3 3 2023-12-11 \N \N \N 1 0 +4 3 2023-12-09 \N \N \N 1 0 + +-- !query15_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query15_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query16_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query16_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query17_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query17_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query18_0_before -- + +-- !query18_0_after -- + +-- !query19_0_before -- +2 3 2023-12-08 20.00 +2 3 2023-12-12 57.40 +2 4 2023-12-10 46.00 + +-- !query19_0_after -- +2 3 2023-12-08 20.00 +2 3 2023-12-12 57.40 +2 4 2023-12-10 46.00 + +-- !query20_0_before -- +2023-12-08 3 2023-12-08 20.00 10.50 9.50 2 0 +2023-12-09 3 2023-12-09 11.50 11.50 11.50 1 0 +2023-12-10 4 2023-12-10 46.00 33.50 12.50 2 0 +2023-12-11 3 2023-12-11 43.20 43.20 43.20 1 0 +2023-12-12 3 2023-12-12 57.40 56.20 1.20 2 0 + +-- !query20_0_after -- +2023-12-08 3 2023-12-08 20.00 10.50 9.50 2 0 +2023-12-09 3 2023-12-09 11.50 11.50 11.50 1 0 +2023-12-10 4 2023-12-10 46.00 33.50 12.50 2 0 +2023-12-11 3 2023-12-11 43.20 43.20 43.20 1 0 +2023-12-12 3 2023-12-12 57.40 56.20 1.20 2 0 + +-- !query21_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 + +-- !query21_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 + +-- !query22_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 +4 3 2023-12-09 11.50 11.50 11.50 1 0 + +-- !query22_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 +4 3 2023-12-09 11.50 11.50 11.50 1 0 + +-- !query23_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 + +-- !query23_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 + +-- !query24_0_before -- +3 2023-12-08 20.00 10.50 9.50 2 0 +3 2023-12-09 11.50 11.50 11.50 1 0 +3 2023-12-11 43.20 43.20 43.20 1 0 +3 2023-12-12 57.40 56.20 1.20 2 0 + +-- !query24_0_after -- +3 2023-12-08 20.00 10.50 9.50 2 0 +3 2023-12-09 11.50 11.50 11.50 1 0 +3 2023-12-11 43.20 43.20 43.20 1 0 +3 2023-12-12 57.40 56.20 1.20 2 0 + +-- !query25_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 +2 3 2023-12-12 57.40 56.20 1.20 2 +2 4 2023-12-10 46.00 33.50 12.50 2 +3 3 2023-12-11 43.20 43.20 43.20 1 +4 3 2023-12-09 11.50 11.50 11.50 1 + +-- !query25_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 +2 3 2023-12-12 57.40 56.20 1.20 2 +2 4 2023-12-10 46.00 33.50 12.50 2 +3 3 2023-12-11 43.20 43.20 43.20 1 +4 3 2023-12-09 11.50 11.50 11.50 1 diff --git a/regression-test/data/nereids_rules_p0/mv/aggregate_without_roll_up.out b/regression-test/data/nereids_rules_p0/mv/aggregate_without_roll_up.out index c866c1795e839f..346814bc083bf7 100644 --- a/regression-test/data/nereids_rules_p0/mv/aggregate_without_roll_up.out +++ b/regression-test/data/nereids_rules_p0/mv/aggregate_without_roll_up.out @@ -1,6 +1,127 @@ -- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- + +-- !query1_0_after -- +1 yy 0 0 77.50 33.50 9.50 5 +2 mi 0 0 57.40 56.20 1.20 2 +2 mm 0 0 43.20 43.20 43.20 1 + +-- !query1_2_before -- +1 yy 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 +2 mi 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 +2 mm 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 + +-- !query1_2_after -- +1 yy 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 +2 mi 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 +2 mm 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 + -- !query2_0_before -- -1 1 yy 0 0 0 0 0 0 0 0 0 0 0 0 0 -1 2 mi 1 0 0 0 0 0 0 0 0 0 0 0 0 -2 2 mm 0 0 0 0 0 0 0 0 0 0 0 0 0 +1 yy 0 0 0 0 0 0 0 0 0 0 0 +2 mi 0 0 0 0 0 0 0 0 0 0 0 +2 mm 0 0 0 0 0 0 0 0 0 0 0 + +-- !query2_0_after -- +1 yy 0 0 0 0 0 0 0 0 0 0 0 +2 mi 0 0 0 0 0 0 0 0 0 0 0 +2 mm 0 0 0 0 0 0 0 0 0 0 0 + +-- !query3_0_before -- +0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 + +-- !query3_0_after -- +0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 + +-- !query13_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 + +-- !query13_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 + +-- !query14_0_before -- +2 3 \N \N \N \N 1 0 +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 4 \N \N \N \N 1 0 +3 3 \N \N \N \N 1 0 +4 3 \N \N \N \N 1 0 + +-- !query14_0_after -- +2 3 \N \N \N \N 1 0 +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 4 \N \N \N \N 1 0 +3 3 \N \N \N \N 1 0 +4 3 \N \N \N \N 1 0 + +-- !query15_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query15_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 +4 3 2023-12-09 11.50 11.50 11.50 1 0 + +-- !query16_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 +4 3 2023-12-09 11.50 11.50 11.50 1 0 + +-- !query16_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 0 +2 3 2023-12-12 57.40 56.20 1.20 2 0 +2 4 2023-12-10 46.00 33.50 12.50 2 0 +3 3 2023-12-11 43.20 43.20 43.20 1 0 +4 3 2023-12-09 11.50 11.50 11.50 1 0 + +-- !query17_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query17_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query18_0_before -- + +-- !query18_0_after -- + +-- !query18_1_before -- + +-- !query18_1_after -- + +-- !query18_2_before -- + +-- !query18_2_after -- + +-- !query19_0_before -- +2 3 2023-12-08 20.00 10.50 9.50 2 +2 3 2023-12-12 57.40 56.20 1.20 2 +2 4 2023-12-10 46.00 33.50 12.50 2 +3 3 2023-12-11 43.20 43.20 43.20 1 +4 3 2023-12-09 11.50 11.50 11.50 1 + +-- !query19_0_after -- +2 3 2023-12-08 20.00 10.50 9.50 2 +2 3 2023-12-12 57.40 56.20 1.20 2 +2 4 2023-12-10 46.00 33.50 12.50 2 +3 3 2023-12-11 43.20 43.20 43.20 1 +4 3 2023-12-09 11.50 11.50 11.50 1 + +-- !query19_1_before -- +4 1 77.50 +4 2 43.20 +6 2 57.40 + +-- !query19_1_after -- +4 1 77.50 +4 2 43.20 +6 2 57.40 diff --git a/regression-test/data/nereids_rules_p0/mv/inner_join.out b/regression-test/data/nereids_rules_p0/mv/inner_join.out index 863d6765698c5c..7ae2e05e523327 100644 --- a/regression-test/data/nereids_rules_p0/mv/inner_join.out +++ b/regression-test/data/nereids_rules_p0/mv/inner_join.out @@ -1,43 +1,265 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !query1_0_before -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_0_after -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_1_before -- 4 4 +4 +4 +6 +6 6 6 -- !query1_1_after -- 4 4 +4 +4 +6 +6 6 6 -- !query1_2_before -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_2_after -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_3_before -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 -- !query1_3_after -- +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 -- !query1_4_before -- -1 -2 +4 +4 +4 +4 +6 +6 +6 +6 -- !query1_4_after -- -1 -2 +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query2_0_before -- + +-- !query2_0_after -- + +-- !query2_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query2_3_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query2_3_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query3_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query3_3_before -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query3_3_after -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query4_0_before -- +4 +4 +4 +4 +4 +4 + +-- !query4_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query5_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !query5_0_after -- +4 +4 +4 +4 +4 +4 +6 +6 -- !query10_0_before -- diff --git a/regression-test/data/nereids_rules_p0/mv/outer_join.out b/regression-test/data/nereids_rules_p0/mv/outer_join.out index 9579dd54810014..94143e800ada97 100644 --- a/regression-test/data/nereids_rules_p0/mv/outer_join.out +++ b/regression-test/data/nereids_rules_p0/mv/outer_join.out @@ -1,69 +1,223 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !query1_0_before -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_0_after -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_1_before -- 4 4 +4 +4 +4 +4 +4 +4 +6 +6 6 6 -- !query1_1_after -- 4 4 +4 +4 +4 +4 +4 +4 +6 +6 6 6 -- !query1_2_before -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_2_after -- 4 +4 +4 +4 +4 +4 +6 6 -- !query1_3_before -- -1 -2 +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 -- !query1_3_after -- -1 -2 +1 1 +1 1 +1 1 +1 1 +1 1 +2 2 +2 2 +2 2 -- !query2_0_before -- -- !query2_0_after -- -- !query2_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 -- !query2_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 --- !query2_2_before -- +-- !query3_0_before -- +4 +4 +4 +4 4 +4 +6 6 --- !query2_2_after -- +-- !query3_0_after -- +4 +4 4 +4 +4 +4 +6 6 --- !query2_3_before -- +-- !query3_1_before -- +4 +4 +4 +4 +4 +4 +6 +6 --- !query2_3_after -- +-- !query3_1_after -- +4 +4 +4 +4 +4 +4 +6 +6 --- !query2_4_before -- +-- !query3_2_before -- +4 +4 +4 +4 +4 +4 +6 +6 --- !query2_4_after -- +-- !query3_2_after -- +4 +4 +4 +4 +4 +4 +6 +6 --- !query2_5_before -- +-- !query4_0_before -- 4 +4 + +-- !query4_0_after -- +4 +4 + +-- !query5_0_before -- +4 +4 +4 +4 +4 +4 +6 6 --- !query2_5_after -- +-- !query5_0_after -- 4 +4 +4 +4 +4 +4 +6 6 +-- !query6_0_before -- +2 3 2023-12-08 +2 3 2023-12-08 +2 3 2023-12-12 +2 4 2023-12-10 +3 3 2023-12-11 +4 3 2023-12-09 + +-- !query6_0_after -- +2 3 2023-12-08 +2 3 2023-12-08 +2 3 2023-12-12 +2 4 2023-12-10 +3 3 2023-12-11 +4 3 2023-12-09 + +-- !query7_0_before -- +3 3 2023-12-11 + +-- !query7_0_after -- +3 3 2023-12-11 + diff --git a/regression-test/suites/nereids_rules_p0/mv/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/aggregate_with_roll_up.groovy index 79a5fe2c1c801b..d6d96acbb8881b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/aggregate_with_roll_up.groovy @@ -31,19 +31,19 @@ suite("aggregate_with_roll_up") { sql """ CREATE TABLE IF NOT EXISTS orders ( - O_ORDERKEY INTEGER NOT NULL, - O_CUSTKEY INTEGER NOT NULL, - O_ORDERSTATUS CHAR(1) NOT NULL, - O_TOTALPRICE DECIMALV3(15,2) NOT NULL, - O_ORDERDATE DATE NOT NULL, - O_ORDERPRIORITY CHAR(15) NOT NULL, - O_CLERK CHAR(15) NOT NULL, - O_SHIPPRIORITY INTEGER NOT NULL, + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, O_COMMENT VARCHAR(79) NOT NULL ) - DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY) - PARTITION BY RANGE(O_ORDERDATE) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) - DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -55,26 +55,26 @@ suite("aggregate_with_roll_up") { sql""" CREATE TABLE IF NOT EXISTS lineitem ( - L_ORDERKEY INTEGER NOT NULL, - L_PARTKEY INTEGER NOT NULL, - L_SUPPKEY INTEGER NOT NULL, - L_LINENUMBER INTEGER NOT NULL, - L_QUANTITY DECIMALV3(15,2) NOT NULL, - L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL, - L_DISCOUNT DECIMALV3(15,2) NOT NULL, - L_TAX DECIMALV3(15,2) NOT NULL, - L_RETURNFLAG CHAR(1) NOT NULL, - L_LINESTATUS CHAR(1) NOT NULL, - L_SHIPDATE DATE NOT NULL, - L_COMMITDATE DATE NOT NULL, - L_RECEIPTDATE DATE NOT NULL, - L_SHIPINSTRUCT CHAR(25) NOT NULL, - L_SHIPMODE CHAR(10) NOT NULL, - L_COMMENT VARCHAR(44) NOT NULL + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL ) - DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) - PARTITION BY RANGE(L_SHIPDATE) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) - DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -86,30 +86,42 @@ suite("aggregate_with_roll_up") { sql """ CREATE TABLE IF NOT EXISTS partsupp ( - PS_PARTKEY INTEGER NOT NULL, - PS_SUPPKEY INTEGER NOT NULL, - PS_AVAILQTY INTEGER NOT NULL, - PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL, - PS_COMMENT VARCHAR(199) NOT NULL + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL ) - DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) - DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3 + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) """ - sql """ insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');""" + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ sql """ - insert into orders values (1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 2, 'ok', 109.2, '2023-12-11', 'c','d',2, 'mm'), - (1, 2, 'ok', 109.2, '2023-12-11', 'c','d',2, 'mi'); + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ sql """ - insert into partsupp values (2, 3, 9, 10.01, 'supply1'), + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), (2, 3, 10, 11.01, 'supply2'); """ @@ -151,87 +163,552 @@ suite("aggregate_with_roll_up") { } } - // select + from + inner join + group by - def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "group by lineitem.L_LINENUMBER, orders.O_CUSTKEY " - def query1_0 = "select lineitem.L_LINENUMBER, sum(O_TOTALPRICE) as sum_alias " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "group by lineitem.L_LINENUMBER" - order_qt_query1_0_before "${query1_0}" - check_rewrite(mv1_0, query1_0, "mv1_0") - order_qt_query1_0_after "${query1_0}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" - - - // select case when + from + where + group by roll up - def mv1_1 = "select O_ORDERDATE, O_SHIPPRIORITY, O_COMMENT, " + - "bitmap_union(to_bitmap(case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_CUSTKEY else null end)) cnt_1, " + - "bitmap_union(to_bitmap(case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_CUSTKEY else null end)) as cnt_2 " + + // single table + // filter + use roll up dimension + def mv1_1 = "select o_orderdate, o_shippriority, o_comment, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, " + + "bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 " + "from orders " + "group by " + - "O_ORDERDATE, " + - "O_SHIPPRIORITY, " + - "O_COMMENT " - def query1_1 = "select O_SHIPPRIORITY, O_COMMENT, " + - "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_CUSTKEY else null end) as cnt_1, " + - "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_CUSTKEY else null end) as cnt_2 " + + "o_orderdate, " + + "o_shippriority, " + + "o_comment " + def query1_1 = "select o_shippriority, o_comment, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + "from orders " + - "where O_ORDERDATE = '2023-12-09' " + + "where o_orderdate = '2023-12-09' " + "group by " + - "O_SHIPPRIORITY, " + - "O_COMMENT " + "o_shippriority, " + + "o_comment " + // rewrite success but cbo not chose, tmp // order_qt_query1_1_before "${query1_1}" -// unsupported, need to fix, because create materialized view contains bitmap which is not supported // check_rewrite(mv1_1, query1_1, "mv1_1") // order_qt_query1_1_after "${query1_1}" // sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" - def mv1_2 = "select O_ORDERDATE, O_SHIPPRIORITY, O_COMMENT, " + - "bitmap_union(to_bitmap(O_CUSTKEY)) as cnt_1, " + - "bitmap_union(to_bitmap(O_CUSTKEY)) as cnt_2 " + + // filter + not use roll up dimension + def mv2_0 = "select o_orderdate, o_shippriority, o_comment, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1, " + + "bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2 " + "from orders " + "group by " + - "O_ORDERDATE, " + - "O_SHIPPRIORITY, " + - "O_COMMENT " - def query1_2 = "select O_SHIPPRIORITY, O_COMMENT, " + - "count(distinct O_CUSTKEY) as cnt_1, " + - "count(distinct O_ORDERKEY) as cnt_2 " + + "o_orderdate, " + + "o_shippriority, " + + "o_comment " + def query2_0 = "select o_shippriority, o_comment, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + "from orders " + - "where O_ORDERDATE = '2023-12-09' " + + "where o_shippriority = 2 " + + "group by " + + "o_shippriority, " + + "o_comment " + // rewrite success but cbo not chose, tmp +// order_qt_query2_0_before "${query2_0}" +// check_rewrite(mv2_0, query2_0, "mv2_0") +// order_qt_query2_0_after "${query2_0}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + // multi table + // filter inside + left + use roll up dimension + def mv13_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query13_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from (select * from lineitem where l_shipdate = '2023-12-11') t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query13_0_before "${query13_0}" + check_rewrite(mv13_0, query13_0, "mv13_0") + order_qt_query13_0_after "${query13_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv13_0""" + + + // filter inside + right + use roll up dimension + def mv14_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query14_0 = "select l_partkey, l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query14_0_before "${query14_0}" + check_rewrite(mv14_0, query14_0, "mv14_0") + order_qt_query14_0_after "${query14_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv14_0""" + + + // filter inside + right + left + use roll up dimension + def mv15_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query15_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from (select * from lineitem where l_shipdate = '2023-12-11') t1 " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query15_0_before "${query15_0}" + check_rewrite(mv15_0, query15_0, "mv15_0") + order_qt_query15_0_after "${query15_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0""" + + + // filter outside + left + use roll up dimension + def mv16_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query16_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_shipdate = '2023-12-11' " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query16_0_before "${query16_0}" + check_rewrite(mv16_0, query16_0, "mv16_0") + order_qt_query16_0_after "${query16_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_0""" + + + // filter outside + right + use roll up dimension + def mv17_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query17_0 = "select t1.l_partkey, t1.l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where o_orderdate = '2023-12-11' " + + "group by " + + "l_shipdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query17_0_before "${query17_0}" + check_rewrite(mv17_0, query17_0, "mv17_0") + order_qt_query17_0_after "${query17_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_0""" + + // filter outside + left + right + use roll up dimension + def mv18_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query18_0 = "select t1.l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where o_orderdate = '2023-12-11' and l_partkey = 2 " + + "group by " + + "l_shipdate, " + + "l_suppkey" + order_qt_query18_0_before "${query18_0}" + check_rewrite(mv18_0, query18_0, "mv18_0") + order_qt_query18_0_after "${query18_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_0""" + + + // filter inside + left + use not roll up dimension + def mv19_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query19_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, sum(o_totalprice) " + + "from (select * from lineitem where l_partkey = 2 ) t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + "group by " + - "O_SHIPPRIORITY, " + - "O_COMMENT " - // order_qt_query1_2_before "${query1_2}" - // unsupported, need to fix - // check_rewrite(mv1_2, query1_2, "mv1_2") - // order_qt_query1_2_after "${query1_2}" - // sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query19_0_before "${query19_0}" + check_rewrite(mv19_0, query19_0, "mv19_0") + order_qt_query19_0_after "${query19_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_0""" + def mv19_1 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from (select * from lineitem where l_partkey = 2) t1 " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query19_1 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end), " + + "count(*) " + + "from (select * from lineitem where l_partkey = 2 and l_suppkey = 3) t1 " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" +// // Should pass but not, tmp +// order_qt_query19_1_before "${query19_1}" +// check_rewrite(mv19_1, query19_1, "mv19_1") +// order_qt_query19_1_after "${query19_1}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_1""" + + + // filter inside + right + use not roll up dimension + def mv20_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query20_0 = "select l_shipdate, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_suppkey" + order_qt_query20_0_before "${query20_0}" + check_rewrite(mv20_0, query20_0, "mv20_0") + order_qt_query20_0_after "${query20_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv20_0""" - // select + from + where + outer join + group by roll up - def mv2_0 = "select L_SHIPDATE, O_ORDERDATE, L_PARTKEY, L_SUPPKEY, sum(O_TOTALPRICE) as sum_total " + + // filter inside + right + left + use not roll up dimension + def mv21_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + "from lineitem " + - "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY and L_SHIPDATE = O_ORDERDATE " + - "group by " + - "L_SHIPDATE, " + - "O_ORDERDATE, " + - "L_PARTKEY, " + - "L_SUPPKEY" - def query2_0 = "select t1.L_PARTKEY, t1.L_SUPPKEY, O_ORDERDATE, sum(O_TOTALPRICE) " + - "from (select * from lineitem where L_SHIPDATE = '2023-12-11') t1 " + - "left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY and t1.L_SHIPDATE = O_ORDERDATE " + - "group by " + - "O_ORDERDATE, " + - "L_PARTKEY, " + - "L_SUPPKEY" - order_qt_query2_0_before "${query2_0}" - check_rewrite(mv2_0, query2_0, "mv2_0") - order_qt_query2_0_after "${query2_0}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query21_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from (select * from lineitem where l_partkey = 2) t1 " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query21_0_before "${query21_0}" + check_rewrite(mv21_0, query21_0, "mv21_0") + order_qt_query21_0_after "${query21_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv21_0""" + + + // filter outside + left + use not roll up dimension + def mv22_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query22_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 2 or l_suppkey = 3 " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query22_0_before "${query22_0}" + check_rewrite(mv22_0, query22_0, "mv22_0") + order_qt_query22_0_after "${query22_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv22_0""" + + + def mv22_1 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "where l_partkey = 2 " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query22_1 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 2 and l_suppkey = 3 " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + // Should pass but not, tmp +// order_qt_query22_1_before "${query22_1}" +// check_rewrite(mv22_1, query22_1, "mv22_1") +// order_qt_query22_1_after "${query22_1}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv22_0""" + + + // filter outside + right + use not roll up dimension + def mv23_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, o_orderstatus, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey, " + + "o_orderstatus" + def query23_0 = "select t1.l_partkey, t1.l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where o_orderdate = '2023-12-08' and o_orderstatus = 'o' " + + "group by " + + "l_shipdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query23_0_before "${query23_0}" + check_rewrite(mv23_0, query23_0, "mv23_0") + order_qt_query23_0_after "${query23_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv23_0""" + + + // filter outside + left + right + not use roll up dimension + def mv24_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) as bitmap_union_basic " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query24_0 = "select t1.l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_suppkey = 3 " + + "group by " + + "l_shipdate, " + + "l_suppkey" + order_qt_query24_0_before "${query24_0}" + check_rewrite(mv24_0, query24_0, "mv24_0") + order_qt_query24_0_after "${query24_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv24_0""" + + + + // without filter + def mv25_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all " + + "from lineitem " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query25_0 = "select l_partkey, l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + + "from lineitem " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query25_0_before "${query25_0}" + check_rewrite(mv25_0, query25_0, "mv25_0") + order_qt_query25_0_after "${query25_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv25_0""" + + // can not rewrite, todo } diff --git a/regression-test/suites/nereids_rules_p0/mv/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/aggregate_without_roll_up.groovy index bc7c2e86b0b76a..055f8d01804a65 100644 --- a/regression-test/suites/nereids_rules_p0/mv/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/aggregate_without_roll_up.groovy @@ -31,19 +31,19 @@ suite("aggregate_without_roll_up") { sql """ CREATE TABLE IF NOT EXISTS orders ( - O_ORDERKEY INTEGER NOT NULL, - O_CUSTKEY INTEGER NOT NULL, - O_ORDERSTATUS CHAR(1) NOT NULL, - O_TOTALPRICE DECIMALV3(15,2) NOT NULL, - O_ORDERDATE DATE NOT NULL, - O_ORDERPRIORITY CHAR(15) NOT NULL, - O_CLERK CHAR(15) NOT NULL, - O_SHIPPRIORITY INTEGER NOT NULL, - O_COMMENT VARCHAR(79) NOT NULL + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + o_comment VARCHAR(79) NOT NULL ) - DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY) - PARTITION BY RANGE(O_ORDERDATE) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) - DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -55,26 +55,26 @@ suite("aggregate_without_roll_up") { sql""" CREATE TABLE IF NOT EXISTS lineitem ( - L_ORDERKEY INTEGER NOT NULL, - L_PARTKEY INTEGER NOT NULL, - L_SUPPKEY INTEGER NOT NULL, - L_LINENUMBER INTEGER NOT NULL, - L_QUANTITY DECIMALV3(15,2) NOT NULL, - L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL, - L_DISCOUNT DECIMALV3(15,2) NOT NULL, - L_TAX DECIMALV3(15,2) NOT NULL, - L_RETURNFLAG CHAR(1) NOT NULL, - L_LINESTATUS CHAR(1) NOT NULL, - L_SHIPDATE DATE NOT NULL, - L_COMMITDATE DATE NOT NULL, - L_RECEIPTDATE DATE NOT NULL, - L_SHIPINSTRUCT CHAR(25) NOT NULL, - L_SHIPMODE CHAR(10) NOT NULL, - L_COMMENT VARCHAR(44) NOT NULL + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL ) - DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) - PARTITION BY RANGE(L_SHIPDATE) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) - DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -86,30 +86,42 @@ suite("aggregate_without_roll_up") { sql """ CREATE TABLE IF NOT EXISTS partsupp ( - PS_PARTKEY INTEGER NOT NULL, - PS_SUPPKEY INTEGER NOT NULL, - PS_AVAILQTY INTEGER NOT NULL, - PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL, - PS_COMMENT VARCHAR(199) NOT NULL + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL ) - DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) - DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3 + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) """ - sql """ insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');""" + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ sql """ - insert into orders values (1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 2, 'ok', 109.2, '2023-12-11', 'c','d',2, 'mm'), - (1, 2, 'ok', 109.2, '2023-12-11', 'c','d',2, 'mi'); + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ sql """ - insert into partsupp values (2, 3, 9, 10.01, 'supply1'), + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), (2, 3, 10, 11.01, 'supply2'); """ @@ -151,22 +163,37 @@ suite("aggregate_without_roll_up") { } } - // select + from + inner join + group by - def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "group by lineitem.L_LINENUMBER, orders.O_CUSTKEY " - def query1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "group by lineitem.L_LINENUMBER, orders.O_CUSTKEY " -// order_qt_query1_0_before "${query1_0}" -// check_rewrite(mv1_0, query1_0, "mv1_0") -// order_qt_query1_0_after "${query1_0}" -// sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + // single table + // with filter + def mv1_0 = "select o_shippriority, o_comment, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2 " + + "from orders " + + "group by " + + "o_shippriority, " + + "o_comment " + def query1_0 = "select o_shippriority, o_comment, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + + "from orders " + + "where o_shippriority in (9.5, 10.5)" + + "group by " + + "o_shippriority, " + + "o_comment " + order_qt_query1_0_before "${query1_0}" + check_rewrite(mv1_0, query1_0, "mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" - // select case when + from + where + group by def mv1_1 = "select O_SHIPPRIORITY, O_COMMENT, " + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + @@ -189,7 +216,7 @@ suite("aggregate_without_roll_up") { "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + "from orders " + - "where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" + + "where O_ORDERDATE < '2023-12-30'" + "group by " + "O_SHIPPRIORITY, " + "O_COMMENT " @@ -214,14 +241,322 @@ suite("aggregate_without_roll_up") { "group by " + "O_SHIPPRIORITY, " + "O_COMMENT " + // should support but not, tmp // order_qt_query1_1_before "${query1_1}" // check_rewrite(mv1_1, query1_1, "mv1_1") // order_qt_query1_1_after "${query1_1}" // sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" - // select case when + from + where + group by - def mv2_0 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " + + def mv1_2 = "select O_SHIPPRIORITY, O_COMMENT, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" + + "group by " + + "O_SHIPPRIORITY, " + + "O_COMMENT " + def query1_2 = "select O_SHIPPRIORITY, O_COMMENT, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" + + "group by " + + "O_SHIPPRIORITY, " + + "O_COMMENT " + order_qt_query1_2_before "${query1_2}" + check_rewrite(mv1_2, query1_2, "mv1_2") + order_qt_query1_2_after "${query1_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" + + // without filter + def mv2_0 = "select O_SHIPPRIORITY, O_COMMENT, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "group by " + + "O_SHIPPRIORITY, " + + "O_COMMENT " + def query2_0 = "select O_SHIPPRIORITY, O_COMMENT, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "group by " + + "O_SHIPPRIORITY, " + + "O_COMMENT " + order_qt_query2_0_before "${query2_0}" + check_rewrite(mv2_0, query2_0, "mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + // without group, scalar aggregate + def mv3_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" + + def query3_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from orders " + + "where O_ORDERDATE < '2023-12-30' and O_ORDERDATE > '2023-12-01'" + order_qt_query3_0_before "${query3_0}" + check_rewrite(mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + + + // multi table + // filter inside + left + def mv13_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query13_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from (select * from lineitem where l_partkey = 2) t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query13_0_before "${query13_0}" + check_rewrite(mv13_0, query13_0, "mv13_0") + order_qt_query13_0_after "${query13_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv13_0""" + + + // filter inside + right + def mv14_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query14_0 = "select l_partkey, l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query14_0_before "${query14_0}" + check_rewrite(mv14_0, query14_0, "mv14_0") + order_qt_query14_0_after "${query14_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv14_0""" + + + // filter inside + right + left + def mv15_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query15_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from (select * from lineitem where l_partkey in (2, 3)) t1 " + + "left join (select * from orders where o_orderstatus = 'o') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query15_0_before "${query15_0}" + check_rewrite(mv15_0, query15_0, "mv15_0") + order_qt_query15_0_after "${query15_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0""" + + // filter outside + left + def mv16_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query16_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey in (1, 2 ,3, 4) " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query16_0_before "${query16_0}" + check_rewrite(mv16_0, query16_0, "mv16_0") + order_qt_query16_0_after "${query16_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_0""" + + + // filter outside + right + def mv17_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query17_0 = "select t1.l_partkey, t1.l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where o_orderdate = '2023-12-11' " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query17_0_before "${query17_0}" + check_rewrite(mv17_0, query17_0, "mv17_0") + order_qt_query17_0_after "${query17_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_0""" + + + def mv17_1 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + @@ -251,7 +586,7 @@ suite("aggregate_without_roll_up") { "lineitem.L_ORDERKEY, " + "orders.O_SHIPPRIORITY, " + "orders.O_COMMENT " - def query2_0 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " + + def query17_1 = "select L_ORDERKEY, O_SHIPPRIORITY, O_COMMENT, " + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + @@ -274,8 +609,160 @@ suite("aggregate_without_roll_up") { "lineitem.L_ORDERKEY, " + "orders.O_SHIPPRIORITY, " + "orders.O_COMMENT " - order_qt_query2_0_before "${query2_0}" - check_rewrite(mv2_0, query2_0, "mv2_0") - order_qt_query2_0_after "${query2_0}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + // rewrite success but cbo not chose, tmp +// order_qt_query17_1_before "${query17_1}" +// check_rewrite(mv17_1, query17_1, "mv17_1") +// order_qt_query17_1_after "${query17_1}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv17_1""" + + // filter outside + left + right + def mv18_0 = "select l_shipdate, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all, " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "l_shipdate, " + + "l_suppkey" + def query18_0 = "select t1.l_suppkey, l_shipdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*), " + + "count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) " + + "from lineitem t1 " + + "left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_shipdate = '2023-12-11' and l_suppkey = 2 " + + "group by " + + "l_shipdate, " + + "l_suppkey" + order_qt_query18_0_before "${query18_0}" + check_rewrite(mv18_0, query18_0, "mv18_0") + order_qt_query18_0_after "${query18_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_0""" + + + def mv18_1 = "select l_linenumber, o_custkey, sum(o_totalprice) as sum_alias " + + "from lineitem " + + "inner join orders on l_orderkey = o_orderkey " + + "group by l_linenumber, o_custkey " + def query18_1 = "select l_linenumber, sum(o_totalprice) as sum_alias " + + "from lineitem " + + "inner join orders on l_orderkey = o_orderkey " + + "where o_custkey = 2 and l_linenumber = 3 " + + "group by l_linenumber, o_custkey " + order_qt_query18_1_before "${query18_1}" + check_rewrite(mv18_1, query18_1, "mv18_1") + order_qt_query18_1_after "${query18_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_1""" + + + def mv18_2 = "select lineitem.l_linenumber, orders.o_custkey, sum(o_totalprice) as sum_alias " + + "from lineitem " + + "inner join orders on l_orderkey = o_orderkey " + + "group by lineitem.l_linenumber, orders.o_custkey " + def query18_2 = "select lineitem.l_linenumber, sum(o_totalprice) as sum_alias " + + "from lineitem " + + "inner join orders on lineitem.l_orderkey = orders.o_orderkey " + + "where o_custkey = 2 and l_suppkey= 4 " + + "group by lineitem.l_linenumber, orders.o_custkey " + order_qt_query18_2_before "${query18_2}" + check_not_match(mv18_2, query18_2, "mv18_2") + order_qt_query18_2_after "${query18_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv18_2""" + + + // without filter + def mv19_0 = "select o_orderdate, l_partkey, l_suppkey, " + + "sum(o_totalprice) as sum_total, " + + "max(o_totalprice) as max_total, " + + "min(o_totalprice) as min_total, " + + "count(*) as count_all " + + "from lineitem " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + def query19_0 = "select l_partkey, l_suppkey, o_orderdate, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + + "from lineitem " + + "left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate " + + "group by " + + "o_orderdate, " + + "l_partkey, " + + "l_suppkey" + order_qt_query19_0_before "${query19_0}" + check_rewrite(mv19_0, query19_0, "mv19_0") + order_qt_query19_0_after "${query19_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_0""" + + + def mv19_1 = "select lineitem.l_linenumber, orders.o_custkey, sum(o_totalprice) as sum_alias " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "group by lineitem.L_LINENUMBER, orders.O_CUSTKEY " + def query19_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "group by lineitem.L_LINENUMBER, orders.O_CUSTKEY " + order_qt_query19_1_before "${query19_1}" + check_rewrite(mv19_1, query19_1, "mv19_1") + order_qt_query19_1_after "${query19_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_1""" + + // without group, scalar aggregate + def mv20_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (7, 9) then O_ORDERSTATUS else null end) as filter_cnt_6, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (11, 13) then O_ORDERSTATUS else null end) as filter_cnt_8, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (12, 11) then O_ORDERSTATUS else null end) as filter_cnt_9, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 15) then O_ORDERSTATUS else null end) as filter_cnt_10, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (11, 12) then O_ORDERSTATUS else null end) as filter_cnt_11, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 25) then O_ORDERSTATUS else null end) as filter_cnt_18, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (19, 3) then O_ORDERSTATUS else null end) as filter_cnt_19, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 20) then O_ORDERSTATUS else null end) as filter_cnt_20 " + + "from lineitem " + + "left join " + + "orders " + + "on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' " + def query20_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (3, 4) then O_ORDERSTATUS else null end) as filter_cnt_3, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (5, 6) then O_ORDERSTATUS else null end) as filter_cnt_4, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (2, 3) then O_ORDERSTATUS else null end) as filter_cnt_5, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (8, 10) then O_ORDERSTATUS else null end) as filter_cnt_7, " + + "count(distinct case when O_SHIPPRIORITY > 4 and O_ORDERKEY IN (3, 6) then O_ORDERSTATUS else null end) as filter_cnt_12, " + + "count(distinct case when O_SHIPPRIORITY > 3 and O_ORDERKEY IN (16, 19) then O_ORDERSTATUS else null end) as filter_cnt_13, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (20, 3) then O_ORDERSTATUS else null end) as filter_cnt_14, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (15, 19) then O_ORDERSTATUS else null end) as filter_cnt_15, " + + "count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (13, 21) then O_ORDERSTATUS else null end) as filter_cnt_16, " + + "count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (14, 22) then O_ORDERSTATUS else null end) as filter_cnt_17 " + + "from lineitem " + + "left join " + + "orders " + + "on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERDATE < '2023-12-30' and orders.O_ORDERDATE > '2023-12-01' " + // rewrite success but cbo not chose, tmp +// order_qt_query20_0_before "${query20_0}" +// check_rewrite(mv20_0, query20_0, "mv20_0") +// order_qt_query20_0_after "${query20_0}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv20_0""" } diff --git a/regression-test/suites/nereids_rules_p0/mv/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/inner_join.groovy index 70bf2f9bcb6ecc..e0c0452415dae7 100644 --- a/regression-test/suites/nereids_rules_p0/mv/inner_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/inner_join.groovy @@ -31,18 +31,19 @@ suite("inner_join") { sql """ CREATE TABLE IF NOT EXISTS orders ( - O_ORDERKEY INTEGER NOT NULL, - O_CUSTKEY INTEGER NOT NULL, - O_ORDERSTATUS CHAR(1) NOT NULL, - O_TOTALPRICE DECIMALV3(15,2) NOT NULL, - O_ORDERDATE DATE NOT NULL, - O_ORDERPRIORITY CHAR(15) NOT NULL, - O_CLERK CHAR(15) NOT NULL, - O_SHIPPRIORITY INTEGER NOT NULL, - O_COMMENT VARCHAR(79) NOT NULL + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + o_comment VARCHAR(79) NOT NULL ) - DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY) - DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -54,25 +55,26 @@ suite("inner_join") { sql""" CREATE TABLE IF NOT EXISTS lineitem ( - L_ORDERKEY INTEGER NOT NULL, - L_PARTKEY INTEGER NOT NULL, - L_SUPPKEY INTEGER NOT NULL, - L_LINENUMBER INTEGER NOT NULL, - L_QUANTITY DECIMALV3(15,2) NOT NULL, - L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL, - L_DISCOUNT DECIMALV3(15,2) NOT NULL, - L_TAX DECIMALV3(15,2) NOT NULL, - L_RETURNFLAG CHAR(1) NOT NULL, - L_LINESTATUS CHAR(1) NOT NULL, - L_SHIPDATE DATE NOT NULL, - L_COMMITDATE DATE NOT NULL, - L_RECEIPTDATE DATE NOT NULL, - L_SHIPINSTRUCT CHAR(25) NOT NULL, - L_SHIPMODE CHAR(10) NOT NULL, - L_COMMENT VARCHAR(44) NOT NULL + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL ) - DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) - DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) @@ -84,29 +86,42 @@ suite("inner_join") { sql """ CREATE TABLE IF NOT EXISTS partsupp ( - PS_PARTKEY INTEGER NOT NULL, - PS_SUPPKEY INTEGER NOT NULL, - PS_AVAILQTY INTEGER NOT NULL, - PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL, - PS_COMMENT VARCHAR(199) NOT NULL + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL ) - DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) - DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3 + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" ) """ - sql """ insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');""" + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ sql """ - insert into orders values (1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 2, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'); + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ sql """ - insert into partsupp values (2, 3, 9, 10.01, 'supply1'), + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), (2, 3, 10, 11.01, 'supply2'); """ @@ -148,7 +163,7 @@ suite("inner_join") { } } - // select + from + inner join + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " @@ -176,7 +191,6 @@ suite("inner_join") { order_qt_query1_1_after "${query1_1}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" - def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from orders " + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " @@ -184,37 +198,220 @@ suite("inner_join") { "from lineitem " + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " order_qt_query1_2_before "${query1_2}" + // join direction is not same, should not match check_rewrite(mv1_2, query1_2, "mv1_2") order_qt_query1_2_after "${query1_2}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" - // select + from + inner join + filter - def mv1_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + // select with complex expression + def mv1_3 = "select l_linenumber, o_custkey " + "from orders " + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query1_3 = "select lineitem.L_LINENUMBER " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "where lineitem.L_LINENUMBER > 10" + def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " + + "case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " + + "from orders " + + "inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY" order_qt_query1_3_before "${query1_3}" check_rewrite(mv1_3, query1_3, "mv1_3") - // tmp annotation, will fix later order_qt_query1_3_after "${query1_3}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" - // select with complex expression + from + inner join - def mv1_4 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + - "from orders " + - "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query1_4 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null " + - "from orders " + - "inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY" + def mv1_4 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query1_4 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" order_qt_query1_4_before "${query1_4}" check_rewrite(mv1_4, query1_4, "mv1_4") order_qt_query1_4_after "${query1_4}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_4""" + // filter outside + left + def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query2_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 10" + order_qt_query2_0_before "${query2_0}" + check_rewrite(mv2_0, query2_0, "mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + order_qt_query2_1_before "${query2_1}" + check_rewrite(mv2_1, query2_1, "mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1""" + + + def mv2_2 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "inner join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1 and l_suppkey = 3" + // Should success but not, because mv contains the part filter of mv, tmp +// order_qt_query2_2_before "${query2_2}" +// check_rewrite(mv2_2, query2_2, "mv2_2") +// order_qt_query2_2_after "${query2_2}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2""" + + + def mv2_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY, l_suppkey " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query2_3= "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " + + "where lineitem.L_LINENUMBER > 1 and l_suppkey = 3 " + order_qt_query2_3_before "${query2_3}" + check_rewrite(mv2_3, query2_3, "mv2_3") + order_qt_query2_3_after "${query2_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_3""" + + + // filter outside + right + def mv3_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_0_before "${query3_0}" + // use a filed not from mv, should not success + check_not_match(mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + + + def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_1_before "${query3_1}" + check_rewrite(mv3_1, query3_1, "mv3_1") + order_qt_query3_1_after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" + + + def mv3_2 = "select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " + + "from lineitem " + + "inner join " + + "(select * from orders where O_ORDERSTATUS = 'o') t2 " + + "on lineitem.L_ORDERKEY = t2.O_ORDERKEY " + def query3_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_2_before "${query3_2}" + check_rewrite(mv3_2, query3_2, "mv3_2") + order_qt_query3_2_after "${query3_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2""" + + + def mv3_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query3_3= "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "inner join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY " + + "where o_custkey in (1, 2, 3, 4) " + order_qt_query3_3_before "${query3_3}" + check_rewrite(mv3_3, query3_3, "mv3_3") + order_qt_query3_3_after "${query3_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_3""" + + + // filter outside + left + right + def mv4_0 = "select l_linenumber, o_custkey, o_orderkey, o_orderstatus " + + "from lineitem " + + "inner join orders on lineitem.l_orderkey = orders.o_orderkey " + def query4_0 = "select lineitem.l_linenumber " + + "from lineitem " + + "inner join orders on lineitem.l_orderkey = orders.o_orderkey " + + "where o_orderstatus = 'o' AND l_linenumber in (1, 2, 3, 4, 5) " + order_qt_query4_0_before "${query4_0}" + check_rewrite(mv4_0, query4_0, "mv4_0") + order_qt_query4_0_after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0""" + + + // filter inside + left + def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 1" + def query5_0 = "select t1.L_LINENUMBER " + + "from (select * from lineitem where l_linenumber > 1) t1 " + + "inner join orders on t1.l_orderkey = orders.O_ORDERKEY " + order_qt_query5_0_before "${query5_0}" + check_rewrite(mv5_0, query5_0, "mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0""" + + + // filter inside + right + def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query6_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from lineitem t1 " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + // should passed but not as isGraphLogicalEquals is false +// order_qt_query6_0_before "${query6_0}" +// check_rewrite(mv6_0, query6_0, "mv6_0") +// order_qt_query6_0_after "${query6_0}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" + + + // filter inside + inner + right + def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query7_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " + + "from lineitem where l_partkey in (3, 4)) t1 " + + "inner join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 3" + // should passed but not, because isGraphLogicalEquals is false +// order_qt_query7_0_before "${query7_0}" +// check_rewrite(mv7_0, query7_0, "mv7_0") +// order_qt_query7_0_after "${query7_0}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" + + // check not match, because use a filed orders.O_SHIPPRIORITY which not in mv def mv10_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + diff --git a/regression-test/suites/nereids_rules_p0/mv/outer_join.groovy b/regression-test/suites/nereids_rules_p0/mv/outer_join.groovy index 08046c20d445f2..d7ec67189ac871 100644 --- a/regression-test/suites/nereids_rules_p0/mv/outer_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/outer_join.groovy @@ -1,4 +1,3 @@ - // 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 @@ -32,21 +31,22 @@ suite("outer_join") { sql """ CREATE TABLE IF NOT EXISTS orders ( - O_ORDERKEY INTEGER NOT NULL, - O_CUSTKEY INTEGER NOT NULL, - O_ORDERSTATUS CHAR(1) NOT NULL, - O_TOTALPRICE DECIMALV3(15,2) NOT NULL, - O_ORDERDATE DATE NOT NULL, - O_ORDERPRIORITY CHAR(15) NOT NULL, - O_CLERK CHAR(15) NOT NULL, - O_SHIPPRIORITY INTEGER NOT NULL, + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, O_COMMENT VARCHAR(79) NOT NULL ) - DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY) - DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" - ) + ); """ sql """ @@ -55,28 +55,29 @@ suite("outer_join") { sql""" CREATE TABLE IF NOT EXISTS lineitem ( - L_ORDERKEY INTEGER NOT NULL, - L_PARTKEY INTEGER NOT NULL, - L_SUPPKEY INTEGER NOT NULL, - L_LINENUMBER INTEGER NOT NULL, - L_QUANTITY DECIMALV3(15,2) NOT NULL, - L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL, - L_DISCOUNT DECIMALV3(15,2) NOT NULL, - L_TAX DECIMALV3(15,2) NOT NULL, - L_RETURNFLAG CHAR(1) NOT NULL, - L_LINESTATUS CHAR(1) NOT NULL, - L_SHIPDATE DATE NOT NULL, - L_COMMITDATE DATE NOT NULL, - L_RECEIPTDATE DATE NOT NULL, - L_SHIPINSTRUCT CHAR(25) NOT NULL, - L_SHIPMODE CHAR(10) NOT NULL, - L_COMMENT VARCHAR(44) NOT NULL + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL ) - DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER) - DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" - ) + ); """ sql """ @@ -85,29 +86,42 @@ suite("outer_join") { sql """ CREATE TABLE IF NOT EXISTS partsupp ( - PS_PARTKEY INTEGER NOT NULL, - PS_SUPPKEY INTEGER NOT NULL, - PS_AVAILQTY INTEGER NOT NULL, - PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL, - PS_COMMENT VARCHAR(199) NOT NULL + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL ) - DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) - DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3 + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" - ) + ); """ - sql """ insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-11', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx');""" + sql """ insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ sql """ - insert into orders values (1, 1, 'ok', 99.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 2, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'); + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); """ sql """ - insert into partsupp values (2, 3, 9, 10.01, 'supply1'), + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), (2, 3, 10, 11.01, 'supply2'); """ @@ -149,7 +163,7 @@ suite("outer_join") { } } - // select + from + left outer join + // without filter def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " @@ -177,7 +191,6 @@ suite("outer_join") { order_qt_query1_1_after "${query1_1}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" - def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from orders " + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " @@ -185,15 +198,17 @@ suite("outer_join") { "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " order_qt_query1_2_before "${query1_2}" + // join direction is not same, should not match check_not_match(mv1_2, query1_2, "mv1_2") order_qt_query1_2_after "${query1_2}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_2""" - // select with complex expression + from + inner join - def mv1_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + // select with complex expression + def mv1_3 = "select l_linenumber, o_custkey " + "from orders " + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null " + + def query1_3 = "select IFNULL(orders.O_CUSTKEY, 0) as custkey_not_null, " + + "case when l_linenumber in (1,2,3) then l_linenumber else o_custkey end as case_when " + "from orders " + "left join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY" order_qt_query1_3_before "${query1_3}" @@ -202,7 +217,7 @@ suite("outer_join") { sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" - // select + from + left outer join + filter + // filter outside + left def mv2_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from orders " + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " @@ -211,22 +226,20 @@ suite("outer_join") { "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + "where lineitem.L_LINENUMBER > 10" order_qt_query2_0_before "${query2_0}" - // should not match, because the join direction is not same check_not_match(mv2_0, query2_0, "mv2_0") order_qt_query2_0_after "${query2_0}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" - def mv2_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + - "from lineitem " + - "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def mv2_1 = "select t1.L_LINENUMBER, orders.O_CUSTKEY " + + "from (select * from lineitem where L_LINENUMBER > 1) t1 " + + "left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " def query2_1 = "select lineitem.L_LINENUMBER " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "where orders.O_ORDERSTATUS = 'ok'" + "where lineitem.L_LINENUMBER > 1" order_qt_query2_1_before "${query2_1}" - // use a filed not from mv, should not success - check_not_match(mv2_1, query2_1, "mv2_1") + check_rewrite(mv2_1, query2_1, "mv2_1") order_qt_query2_1_after "${query2_1}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1""" @@ -237,52 +250,115 @@ suite("outer_join") { def query2_2 = "select lineitem.L_LINENUMBER " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "where lineitem.L_LINENUMBER > 1" - order_qt_query2_2_before "${query2_2}" - check_rewrite(mv2_2, query2_2, "mv2_2") - order_qt_query2_2_after "${query2_2}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2""" + "where lineitem.L_LINENUMBER > 1 and l_suppkey = 3" + // Should success but not, tmp +// order_qt_query2_2_before "${query2_2}" +// check_rewrite(mv2_2, query2_2, "mv2_2") +// order_qt_query2_2_after "${query2_2}" +// sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_2""" - def mv2_3 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " + + // filter outside + right + def mv3_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query2_3 = "select lineitem.L_LINENUMBER " + + def query3_0 = "select lineitem.L_LINENUMBER " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "where orders.O_ORDERSTATUS = 'ok'" - order_qt_query2_3_before "${query2_3}" - // left outer -> inner jon because orders.O_ORDERSTATUS = 'ok', but mv is left join, will fix in the future -// check_rewrite(mv2_3, query2_3, "mv2_3") - order_qt_query2_3_after "${query2_3}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_3""" + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_0_before "${query3_0}" + // use a filed not from mv, should not success + check_not_match(mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" - def mv2_4 = "select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " + + def mv3_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, orders.O_ORDERSTATUS " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query3_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_1_before "${query3_1}" + check_rewrite(mv3_1, query3_1, "mv3_1") + order_qt_query3_1_after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" + + + def mv3_2 = "select lineitem.L_LINENUMBER, t2.O_CUSTKEY, t2.O_ORDERSTATUS " + "from lineitem " + "left join " + - "(select * from orders where O_ORDERSTATUS = 'ok') t2 " + + "(select * from orders where O_ORDERSTATUS = 'o') t2 " + "on lineitem.L_ORDERKEY = t2.O_ORDERKEY " - def query2_4 = "select lineitem.L_LINENUMBER " + + def query3_2 = "select lineitem.L_LINENUMBER " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "where orders.O_ORDERSTATUS = 'ok'" - order_qt_query2_4_before "${query2_4}" + "where orders.O_ORDERSTATUS = 'o'" + order_qt_query3_2_before "${query3_2}" // should not success, as mv filter is under left outer input - check_not_match(mv2_4, query2_4, "mv2_4") - order_qt_query2_4_after "${query2_4}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_4""" + check_not_match(mv3_2, query3_2, "mv3_2") + order_qt_query3_2_after "${query3_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2""" + + + // filter outside + left + right + def mv4_0 = "select l_linenumber, o_custkey, o_orderkey, o_orderstatus " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey " + def query4_0 = "select lineitem.l_linenumber " + + "from lineitem " + + "left join orders on lineitem.l_orderkey = orders.o_orderkey " + + "where o_orderstatus = 'o' AND o_orderkey = 1" + order_qt_query4_0_before "${query4_0}" + check_rewrite(mv4_0, query4_0, "mv4_0") + order_qt_query4_0_after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0""" - def mv2_5 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + // filter inside + left + def mv5_0 = "select lineitem.l_linenumber, orders.o_custkey " + "from lineitem " + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + "where lineitem.L_LINENUMBER > 1" - def query2_5 = "select t1.L_LINENUMBER " + - "from (select * from lineitem where L_LINENUMBER > 1) t1 " + - "left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " - order_qt_query2_5_before "${query2_5}" - check_rewrite(mv2_5, query2_5, "mv2_5") - order_qt_query2_5_after "${query2_5}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_5""" + def query5_0 = "select t1.L_LINENUMBER " + + "from (select * from lineitem where l_linenumber > 1) t1 " + + "left join orders on t1.l_orderkey = orders.O_ORDERKEY " + order_qt_query5_0_before "${query5_0}" + check_rewrite(mv5_0, query5_0, "mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0""" + + + // filter inside + right + def mv6_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query6_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from lineitem t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + order_qt_query6_0_before "${query6_0}" + check_rewrite(mv6_0, query6_0, "mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" + + + // filter inside + left + right + def mv7_0 = "select l_shipdate, o_orderdate, l_partkey, l_suppkey " + + "from lineitem " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate " + def query7_0 = "select l_partkey, l_suppkey, l_shipdate " + + "from (select l_shipdate, l_orderkey, l_partkey, l_suppkey " + + "from lineitem where l_partkey in (3, 4)) t1 " + + "left join (select * from orders where o_orderdate = '2023-12-08') t2 " + + "on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate " + + "where l_partkey = 3" + order_qt_query7_0_before "${query7_0}" + check_rewrite(mv7_0, query7_0, "mv7_0") + order_qt_query7_0_after "${query7_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" + } diff --git a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy new file mode 100644 index 00000000000000..98b205695a7b41 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy @@ -0,0 +1,202 @@ +// 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("partition_mv_rewrite") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "SET enable_nereids_planner=true" + sql "SET enable_fallback_to_original_planner=false" + sql "SET enable_materialized_view_rewrite=true" + sql "SET enable_nereids_timeout = false" + // tmp disable to rewrite, will be removed in the future + sql "SET disable_nereids_rules = 'INFER_PREDICATES, ELIMINATE_OUTER_JOIN'" + + sql """ + drop table if exists orders + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders ( + o_orderkey integer not null, + o_custkey integer not null, + o_orderstatus char(1) not null, + o_totalprice decimalv3(15,2) not null, + o_orderdate date not null, + o_orderpriority char(15) not null, + o_clerk char(15) not null, + o_shippriority integer not null, + o_comment varchar(79) not null + ) + DUPLICATE KEY(o_orderkey, o_custkey) + PARTITION BY RANGE(o_orderdate)( + FROM ('2023-10-17') TO ('2023-10-20') INTERVAL 1 DAY + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem ( + l_orderkey integer not null, + l_partkey integer not null, + l_suppkey integer not null, + l_linenumber integer not null, + l_quantity decimalv3(15,2) not null, + l_extendedprice decimalv3(15,2) not null, + l_discount decimalv3(15,2) not null, + l_tax decimalv3(15,2) not null, + l_returnflag char(1) not null, + l_linestatus char(1) not null, + l_shipdate date not null, + l_commitdate date not null, + l_receiptdate date not null, + l_shipinstruct char(25) not null, + l_shipmode char(10) not null, + l_comment varchar(44) not null + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + PARTITION BY RANGE(l_shipdate) + (FROM ('2023-10-17') TO ('2023-10-20') INTERVAL 1 DAY) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql""" + insert into orders values + (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), + (2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'), + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'); + """ + + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), + (2, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-10-19', '2023-10-19', '2023-10-19', 'c', 'd', 'xxxxxxxxx'); + """ + + + def mv_sql_def = "select l_shipdate, o_orderdate, l_partkey,\n" + + "l_suppkey, sum(o_totalprice) as sum_total\n" + + "from lineitem\n" + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate\n" + + "group by\n" + + "l_shipdate,\n" + + "o_orderdate,\n" + + "l_partkey,\n" + + "l_suppkey;" + + sql """ + CREATE MATERIALIZED VIEW mv1_0 + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_sql_def} + """ + + def all_partition_sql = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total\n" + + "from lineitem\n" + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate\n" + + "group by\n" + + "l_shipdate,\n" + + "o_orderdate,\n" + + "l_partkey,\n" + + "l_suppkey;" + + + def partition_sql = "select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total\n" + + "from lineitem\n" + + "left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate\n" + + "where (l_shipdate>= '2023-10-18' and l_shipdate <= '2023-10-19')\n" + + "group by\n" + + "l_shipdate,\n" + + "o_orderdate,\n" + + "l_partkey,\n" + + "l_suppkey;" + + def check_rewrite = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + contains "(${mv_name})" + } + } + + def check_not_match = { mv_sql, query_sql, mv_name -> + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS ${mv_sql} + """ + + def job_name = getJobName(db, mv_name); + waitingMTMVTaskFinished(job_name) + explain { + sql("${query_sql}") + notContains "(${mv_name})" + } + } + + waitingMTMVTaskFinished(getJobName(db, "mv1_0")) + + check_rewrite(mv_sql_def, all_partition_sql, "mv1_0") + check_rewrite(mv_sql_def, partition_sql, "mv1_0") + + // partition is invalid, so can use partition 2023-10-17 to rewrite + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'); + """ + + // wait partition is invalid + sleep(3000) + check_not_match(mv_sql_def, all_partition_sql, "mv1_0") + check_rewrite(mv_sql_def, partition_sql, "mv1_0") + + sql """ + Refresh MATERIALIZED VIEW mv1_0; + """ + waitingMTMVTaskFinished(getJobName(db, "mv1_0")) + + check_rewrite(mv_sql_def, all_partition_sql, "mv1_0") + check_rewrite(mv_sql_def, partition_sql, "mv1_0") +}