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 a2c87581fc8885..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; @@ -128,10 +129,6 @@ public MTMVRelation getRelation() { return relation; } - public MTMVCache getCache() { - return cache; - } - public void setCache(MTMVCache cache) { this.cache = cache; } @@ -193,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/MTMVCache.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java index e4aa36d9b61e49..07fda55e49413d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVCache.java @@ -74,7 +74,7 @@ public static MTMVCache from(MTMV mtmv, ConnectContext connectContext) { ? (Plan) ((LogicalResultSink) mvRewrittenPlan).child() : mvRewrittenPlan; // use rewritten plan output expression currently, if expression rewrite fail, // consider to use the analyzed plan for output expressions only - List mvOutputExpressions = mvRewrittenPlan.getExpressions().stream() + List mvOutputExpressions = mvPlan.getExpressions().stream() .map(NamedExpression.class::cast) .collect(Collectors.toList()); return new MTMVCache(mvPlan, mvOutputExpressions); 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 92149b3b46530e..11da5104255fc3 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 @@ -30,6 +30,7 @@ import org.apache.doris.nereids.trees.plans.commands.info.TableNameInfo; import org.apache.doris.persist.AlterMTMV; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; @@ -49,7 +50,7 @@ public class MTMVRelationManager implements MTMVHookService { private Map> tableMTMVs = Maps.newConcurrentMap(); public Set getMtmvsByBaseTable(BaseTableInfo table) { - return tableMTMVs.get(table); + return tableMTMVs.getOrDefault(table, ImmutableSet.of()); } public Set getAvailableMTMVs(List tableInfos) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java index 81da2946ff1223..23dfb9d5bccca3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java @@ -253,7 +253,7 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne List res = Lists.newArrayList(); Collection allPartitions = mtmv.getPartitions(); // check session variable if enable rewrite - if (!ctx.getSessionVariable().isEnableMvRewrite()) { + if (!ctx.getSessionVariable().isEnableMaterializedViewRewrite()) { return res; } MTMVRelation mtmvRelation = mtmv.getRelation(); @@ -438,7 +438,7 @@ private static long getTableMinVisibleVersionTime(OlapTable table) { * @param relatedTable * @return mv.partitionId ==> relatedTable.partitionId */ - private static Map> getMvToBasePartitions(MTMV mtmv, OlapTable relatedTable) + public static Map> getMvToBasePartitions(MTMV mtmv, OlapTable relatedTable) throws AnalysisException { HashMap> res = Maps.newHashMap(); Map relatedTableItems = relatedTable.getPartitionInfo().getIdToItem(false); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 88472e30bc4a12..885d43f0ac9352 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -318,7 +318,9 @@ public void setOuterScope(@Nullable Scope outerScope) { } public List getMaterializationContexts() { - return materializationContexts; + return materializationContexts.stream() + .filter(MaterializationContext::isAvailable) + .collect(Collectors.toList()); } public void addMaterializationContext(MaterializationContext materializationContext) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/PlaceholderExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/PlaceholderExpression.java index 8f069b25694f95..b2acdddd64fec5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/PlaceholderExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/PlaceholderExpression.java @@ -33,7 +33,6 @@ * @see PlaceholderCollector */ public class PlaceholderExpression extends Expression implements AlwaysNotNullable { - private final Class delegateClazz; /** * 1 based 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 a38c33b76b27b2..a9ce2cce17412d 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 @@ -40,7 +40,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; @@ -222,7 +226,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 ca2981d2785528..685f8a8c3a9eab 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 @@ -25,12 +25,16 @@ import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanSplitContext; import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping; import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.expressions.Any; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; -import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnion; +import org.apache.doris.nereids.trees.expressions.functions.agg.CouldRollUp; +import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -39,8 +43,11 @@ import com.google.common.collect.HashMultimap; 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; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -53,6 +60,17 @@ */ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMaterializedViewRule { + 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(new Count(true, Any.INSTANCE), + new BitmapUnion(Any.INSTANCE)); + } + @Override protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, @@ -63,10 +81,12 @@ protected Plan rewriteQueryByView(MatchMode matchMode, // get view and query aggregate and top plan correspondingly Pair> viewTopPlanAndAggPair = splitToTopPlanAndAggregate(viewStructInfo); if (viewTopPlanAndAggPair == null) { + logger.warn(currentClassName + " split to view to top plan and agg fail so return null"); return null; } Pair> queryTopPlanAndAggPair = splitToTopPlanAndAggregate(queryStructInfo); if (queryTopPlanAndAggPair == null) { + logger.warn(currentClassName + " split to query to top plan and agg fail so return null"); return null; } // Firstly, handle query group by expression rewrite @@ -88,13 +108,14 @@ protected Plan rewriteQueryByView(MatchMode matchMode, needRollUp = !queryGroupShuttledExpression.equals(viewGroupShuttledExpression); } if (!needRollUp) { - List rewrittenQueryGroupExpr = rewriteExpression(queryTopPlan.getOutput(), + List rewrittenQueryGroupExpr = rewriteExpression(queryTopPlan.getExpressions(), queryTopPlan, materializationContext.getMvExprToMvScanExprMapping(), queryToViewSlotMapping, true); - if (rewrittenQueryGroupExpr == null) { + if (rewrittenQueryGroupExpr.isEmpty()) { // can not rewrite, bail out. + logger.debug(currentClassName + " can not rewrite expression when not need roll up"); return null; } return new LogicalProject<>( @@ -109,12 +130,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.debug(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.warn(currentClassName + " query top plan split to group by and function fail so return null"); return null; } // Secondly, try to roll up the agg functions @@ -132,23 +155,19 @@ protected Plan rewriteQueryByView(MatchMode matchMode, for (Expression topExpression : queryTopPlan.getExpressions()) { // is agg function, try to roll up and rewrite if (queryTopPlanFunctionSet.contains(topExpression)) { - Expression needRollupShuttledExpr = ExpressionUtils.shuttleExpressionWithLineage( + Expression queryFunctionShuttled = ExpressionUtils.shuttleExpressionWithLineage( topExpression, queryTopPlan); - if (!mvExprToMvScanExprQueryBased.containsKey(needRollupShuttledExpr)) { - // function can not rewrite by view - return null; - } // try to roll up - AggregateFunction needRollupAggFunction = (AggregateFunction) topExpression.firstMatch( + AggregateFunction queryFunction = (AggregateFunction) topExpression.firstMatch( expr -> expr instanceof AggregateFunction); - AggregateFunction rollupAggregateFunction = rollup(needRollupAggFunction, - mvExprToMvScanExprQueryBased.get(needRollupShuttledExpr)); + Function rollupAggregateFunction = rollup(queryFunction, queryFunctionShuttled, + mvExprToMvScanExprQueryBased); if (rollupAggregateFunction == null) { return null; } // key is query need roll up expr, value is mv scan based roll up expr - needRollupExprMap.put(needRollupShuttledExpr, rollupAggregateFunction); + needRollupExprMap.put(queryFunctionShuttled, rollupAggregateFunction); // rewrite query function expression by mv expression Expression rewrittenFunctionExpression = rewriteExpression(topExpression, queryTopPlan, @@ -156,6 +175,7 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryToViewSlotMapping, false); if (rewrittenFunctionExpression == null) { + logger.debug(currentClassName + " roll up expression can not rewrite by view so return null"); return null; } finalAggregateExpressions.add((NamedExpression) rewrittenFunctionExpression); @@ -165,6 +185,8 @@ protected Plan rewriteQueryByView(MatchMode matchMode, ExpressionUtils.shuttleExpressionWithLineage(topExpression, queryTopPlan); if (!mvExprToMvScanExprQueryBased.containsKey(queryGroupShuttledExpr)) { // group expr can not rewrite by view + logger.debug(currentClassName + + " view group expressions can not contains the query group by expression so return null"); return null; } groupRewrittenExprMap.put(queryGroupShuttledExpr, @@ -177,6 +199,8 @@ protected Plan rewriteQueryByView(MatchMode matchMode, queryToViewSlotMapping, true); if (rewrittenGroupExpression == null) { + logger.debug(currentClassName + + " query top expression can not be rewritten by view so return null"); return null; } finalAggregateExpressions.add((NamedExpression) rewrittenGroupExpression); @@ -226,17 +250,33 @@ protected Plan rewriteQueryByView(MatchMode matchMode, } // only support sum roll up, support other agg functions later. - private AggregateFunction rollup(AggregateFunction originFunction, - Expression mappedExpression) { - Class rollupAggregateFunction = originFunction.getRollup(); - if (rollupAggregateFunction == null) { + private Function rollup(AggregateFunction queryFunction, + Expression queryFunctionShuttled, + Map mvExprToMvScanExprQueryBased) { + if (!(queryFunction instanceof CouldRollUp)) { return null; } - if (Sum.class.isAssignableFrom(rollupAggregateFunction)) { - return new Sum(originFunction.isDistinct(), mappedExpression); + Expression rollupParam = null; + if (mvExprToMvScanExprQueryBased.containsKey(queryFunctionShuttled)) { + // function can rewrite by view + rollupParam = mvExprToMvScanExprQueryBased.get(queryFunctionShuttled); + } else { + // function can not rewrite by view, try to use complex roll up param + // eg: query is count(distinct param), mv sql is bitmap_union(to_bitmap(param)) + for (Expression mvExprShuttled : mvExprToMvScanExprQueryBased.keySet()) { + if (!(mvExprShuttled instanceof Function)) { + continue; + } + if (isAggregateFunctionEquivalent(queryFunction, (Function) mvExprShuttled)) { + rollupParam = mvExprToMvScanExprQueryBased.get(mvExprShuttled); + } + } } - // can rollup return null - return null; + if (rollupParam == null) { + return null; + } + // do roll up + return ((CouldRollUp) queryFunction).constructRollUp(rollupParam); } private Pair, Set> topPlanSplitToGroupAndFunction( @@ -306,4 +346,23 @@ protected boolean checkPattern(StructInfo structInfo) { } return true; } + + private boolean isAggregateFunctionEquivalent(Function queryFunction, Function viewFunction) { + if (queryFunction.equals(viewFunction)) { + return true; + } + // get query equivalent function + Expression equivalentFunction = null; + for (Map.Entry entry : AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP.entrySet()) { + if (entry.getKey().equals(queryFunction)) { + equivalentFunction = entry.getValue(); + } + } + // check is have equivalent function or not + if (equivalentFunction == null) { + return false; + } + // current compare + return equivalentFunction.equals(viewFunction); + } } 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..7825467628a2f3 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.warn(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 483e8e517a05e6..4e532366a33b40 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 @@ -17,8 +17,21 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.PartitionInfo; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVCache; +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.ExplorationRuleFactory; 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; @@ -36,26 +49,32 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; 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.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; /** * The abstract class for all materialized view rules */ -public abstract class AbstractMaterializedViewRule { - +public abstract class AbstractMaterializedViewRule implements ExplorationRuleFactory { 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()); /** * The abstract template method for query rewrite, it contains the main logic and different query @@ -65,6 +84,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; } @@ -72,6 +92,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; } @@ -80,40 +101,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; } - Plan mvPlan = materializationContext.getMtmv().getCache().getLogicalPlan(); - List viewStructInfos = extractStructInfo(mvPlan, cascadesContext); + MTMV mtmv = materializationContext.getMTMV(); + MTMVCache mtmvCache = getCacheFromMTMV(mtmv, cascadesContext); + if (mtmvCache == 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 @@ -124,6 +156,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; @@ -139,6 +172,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); @@ -151,14 +185,108 @@ 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); } } return rewriteResults; } + /** + * Partition will be pruned in query then add the record the partitions to select partitions on + * catalog relation. + * Maybe only just some partitions is valid in materialized view, so we should check if the mv can + * offer the partitions which query used or not. + */ + protected boolean checkPartitionIsValid( + StructInfo queryInfo, + MaterializationContext materializationContext, + CascadesContext cascadesContext) { + // check partition is valid or not + MTMV mtmv = materializationContext.getMTMV(); + PartitionInfo mvPartitionInfo = mtmv.getPartitionInfo(); + if (PartitionType.UNPARTITIONED.equals(mvPartitionInfo.getType())) { + // if not partition, if rewrite success, it means mv is available + return true; + } + // check mv related table partition is valid or not + MTMVPartitionInfo mvCustomPartitionInfo = mtmv.getMvPartitionInfo(); + BaseTableInfo relatedPartitionTable = mvCustomPartitionInfo.getRelatedTable(); + if (relatedPartitionTable == null) { + return true; + } + Optional relatedTableRelation = queryInfo.getRelations().stream() + .filter(LogicalOlapScan.class::isInstance) + .filter(relation -> relatedPartitionTable.equals(new BaseTableInfo(relation.getTable()))) + .map(LogicalOlapScan.class::cast) + .findFirst(); + if (!relatedTableRelation.isPresent()) { + logger.warn("mv is partition update, but related table relation is null"); + return false; + } + OlapTable relatedTable = relatedTableRelation.get().getTable(); + Map> mvToBasePartitionMap; + try { + mvToBasePartitionMap = MTMVUtil.getMvToBasePartitions(mtmv, relatedTable); + } catch (AnalysisException e) { + logger.warn("mvRewriteSuccess getMvToBasePartitions fail", e); + return false; + } + // get mv valid partitions + Collection mvDataValidPartitions = MTMVUtil.getMTMVCanRewritePartitions(mtmv, + cascadesContext.getConnectContext()); + Map allPartitions = mvPartitionInfo.getAllPartitions(); + if (!allPartitions.isEmpty() && mvDataValidPartitions.isEmpty()) { + // do not have valid partition + return false; + } + // get mv related table valid partitions + Set relatedTalbeValidSet = mvDataValidPartitions.stream() + .map(partition -> { + Set relatedBaseTablePartitions = mvToBasePartitionMap.get(partition.getId()); + if (relatedBaseTablePartitions == null || relatedBaseTablePartitions.isEmpty()) { + return ImmutableList.of(); + } else { + return relatedBaseTablePartitions; + } + }) + .flatMap(Collection::stream) + .map(Long.class::cast) + .collect(Collectors.toSet()); + // get query selected partitions to make the partitions is valid or not + Set relatedTableSelectedPartitionToCheck = + new HashSet<>(relatedTableRelation.get().getSelectedPartitionIds()); + if (relatedTableSelectedPartitionToCheck.isEmpty()) { + relatedTableSelectedPartitionToCheck.addAll(relatedTable.getPartitionIds()); + } + return !relatedTalbeValidSet.isEmpty() + && relatedTalbeValidSet.containsAll(relatedTableSelectedPartitionToCheck); + } + + private MTMVCache getCacheFromMTMV(MTMV mtmv, CascadesContext cascadesContext) { + MTMVCache cache; + try { + cache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext()); + } catch (AnalysisException analysisException) { + logger.warn(this.getClass().getSimpleName() + " get mtmv cache analysisException", analysisException); + return null; + } + return cache; + } + /** * Rewrite query by view, for aggregate or join rewriting should be different inherit class implementation */ @@ -268,6 +396,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<>(); @@ -276,6 +405,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 = @@ -283,6 +413,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 @@ -330,6 +461,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); @@ -349,6 +481,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/InitMaterializationContextHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java index b6c7234d5bff23..07a4ff208bca18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/InitMaterializationContextHook.java @@ -19,12 +19,8 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MTMV; -import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.TableIf.TableType; -import org.apache.doris.common.MetaNotFoundException; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.mtmv.MTMVRelationManager; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.PlannerHook; @@ -42,13 +38,14 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; -/** If enable query rewrite with mv, should init materialization context after analyze*/ +/** + * If enable query rewrite with mv, should init materialization context after analyze + */ public class InitMaterializationContextHook implements PlannerHook { public static final Logger LOG = LogManager.getLogger(InitMaterializationContextHook.class); @@ -60,7 +57,9 @@ public void afterAnalyze(NereidsPlanner planner) { } private void initMaterializationContext(CascadesContext cascadesContext) { - + if (!cascadesContext.getConnectContext().getSessionVariable().isEnableMaterializedViewRewrite()) { + return; + } Plan rewritePlan = cascadesContext.getRewritePlan(); TableCollectorContext collectorContext = new TableCollectorContext(Sets.newHashSet()); rewritePlan.accept(TableCollector.INSTANCE, collectorContext); @@ -68,48 +67,30 @@ private void initMaterializationContext(CascadesContext cascadesContext) { if (collectedTables.isEmpty()) { return; } - List baseTableUsed = + List usedBaseTables = collectedTables.stream().map(BaseTableInfo::new).collect(Collectors.toList()); - // TODO the logic should be move to MTMVRelationManager later when getAvailableMaterializedView is ready in - // MV Cache manager - Env env = cascadesContext.getConnectContext().getEnv(); - MTMVRelationManager cacheManager = env.getMtmvService().getRelationManager(); - Set materializedViews = new HashSet<>(); - for (BaseTableInfo baseTableInfo : baseTableUsed) { - Set mtmvsByBaseTable = cacheManager.getMtmvsByBaseTable(baseTableInfo); - if (mtmvsByBaseTable == null || mtmvsByBaseTable.isEmpty()) { - continue; - } - materializedViews.addAll(mtmvsByBaseTable); - } - if (materializedViews.isEmpty()) { + Set availableMTMVs = Env.getCurrentEnv().getMtmvService().getRelationManager() + .getAvailableMTMVs(usedBaseTables); + if (availableMTMVs.isEmpty()) { return; } - materializedViews.forEach(mvBaseTableInfo -> { - try { - MTMV materializedView = (MTMV) Env.getCurrentInternalCatalog() - .getDbOrMetaException(mvBaseTableInfo.getDbId()) - .getTableOrMetaException(mvBaseTableInfo.getTableId(), TableType.MATERIALIZED_VIEW); - - // generate outside, maybe add partition filter in the future - LogicalOlapScan mvScan = new LogicalOlapScan( - cascadesContext.getStatementContext().getNextRelationId(), - (OlapTable) materializedView, - ImmutableList.of(materializedView.getQualifiedDbName()), - // this must be empty, or it will be used to sample - Lists.newArrayList(), - Lists.newArrayList(), - Optional.empty()); - mvScan = mvScan.withMaterializedIndexSelected(PreAggStatus.on(), materializedView.getBaseIndexId()); - List mvProjects = mvScan.getOutput().stream().map(NamedExpression.class::cast) - .collect(Collectors.toList()); - // todo should force keep consistency to mv sql plan output - Plan projectScan = new LogicalProject(mvProjects, mvScan); - cascadesContext.addMaterializationContext( - MaterializationContext.fromMaterializedView(materializedView, projectScan, cascadesContext)); - } catch (MetaNotFoundException metaNotFoundException) { - LOG.error(mvBaseTableInfo.toString() + " can not find corresponding materialized view."); - } + availableMTMVs.forEach(materializedView -> { + // generate outside, maybe add partition filter in the future + LogicalOlapScan mvScan = new LogicalOlapScan( + cascadesContext.getStatementContext().getNextRelationId(), + materializedView, + ImmutableList.of(materializedView.getQualifiedDbName()), + // this must be empty, or it will be used to sample + Lists.newArrayList(), + Lists.newArrayList(), + Optional.empty()); + mvScan = mvScan.withMaterializedIndexSelected(PreAggStatus.on(), materializedView.getBaseIndexId()); + List mvProjects = mvScan.getOutput().stream().map(NamedExpression.class::cast) + .collect(Collectors.toList()); + // todo should force keep consistency to mv sql plan output + Plan projectScan = new LogicalProject(mvProjects, mvScan); + cascadesContext.addMaterializationContext( + MaterializationContext.fromMaterializedView(materializedView, projectScan, cascadesContext)); }); } } 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 4f0b63d2ee7981..f61de5cc2b5a99 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 @@ -19,6 +19,7 @@ import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; import org.apache.doris.mtmv.MTMVCache; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.memo.GroupId; @@ -27,6 +28,8 @@ import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.HashSet; import java.util.List; @@ -37,6 +40,7 @@ */ public class MaterializationContext { + private static final Logger LOG = LogManager.getLogger(MaterializationContext.class); private MTMV mtmv; // Should use stmt id generator in query context private final Plan mvScanPlan; @@ -46,6 +50,7 @@ public class MaterializationContext { private final Set matchedGroups = new HashSet<>(); // generate form mv scan plan private ExpressionMapping mvExprToMvScanExprMapping; + private boolean available = true; /** * MaterializationContext, this contains necessary info for query rewriting by mv @@ -59,11 +64,16 @@ public MaterializationContext(MTMV mtmv, this.mvScanPlan = mvScanPlan; this.baseTables = baseTables; this.baseViews = baseViews; - MTMVCache mtmvCache = mtmv.getCache(); - // TODO This logic should move to materialized view cache manager + + MTMVCache mtmvCache = null; + try { + mtmvCache = mtmv.getOrGenerateCache(cascadesContext.getConnectContext()); + } catch (AnalysisException e) { + LOG.warn("MaterializationContext init mv cache generate fail", e); + } if (mtmvCache == null) { - mtmvCache = mtmvCache.from(mtmv, cascadesContext.getConnectContext()); - mtmv.setCache(mtmvCache); + this.available = false; + return; } // mv output expression shuttle, this will be used to expression rewrite this.mvExprToMvScanExprMapping = ExpressionMapping.generate( @@ -85,7 +95,7 @@ public void addMatchedGroup(GroupId groupId) { matchedGroups.add(groupId); } - public MTMV getMtmv() { + public MTMV getMTMV() { return mtmv; } @@ -105,6 +115,10 @@ public ExpressionMapping getMvExprToMvScanExprMapping() { return mvExprToMvScanExprMapping; } + public boolean isAvailable() { + return available; + } + /** * MaterializationContext fromMaterializedView */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java index 3bd0b1080cc9fe..2674bf2bf22539 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java @@ -18,9 +18,7 @@ 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.LogicalAggregate; @@ -31,7 +29,7 @@ /** * This is responsible for aggregate rewriting according to different pattern * */ -public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule implements RewriteRuleFactory { +public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule { public static final MaterializedViewAggregateRule INSTANCE = new MaterializedViewAggregateRule(); @@ -41,6 +39,6 @@ public List buildRules() { logicalAggregate(any()).thenApplyMulti(ctx -> { LogicalAggregate root = ctx.root; return rewrite(root, ctx.cascadesContext); - }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_AGGREGATE, RulePromise.EXPLORE)); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_AGGREGATE)); } } 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..04d5546b0a988e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterJoinRule.java @@ -0,0 +1,45 @@ +// 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.RuleType; +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 { + + 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)); + } +} 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..4ebee36585d88c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewFilterProjectJoinRule.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.RuleType; +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 { + + 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)); + } +} 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..cf2b246cd1dfd0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewOnlyJoinRule.java @@ -0,0 +1,44 @@ +// 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.RuleType; +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 { + + 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)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectAggregateRule.java index e9a31f45535c84..f76ec3d26aff95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectAggregateRule.java @@ -18,9 +18,7 @@ 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.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -30,8 +28,7 @@ import java.util.List; /**MaterializedViewProjectAggregateRule*/ -public class MaterializedViewProjectAggregateRule extends AbstractMaterializedViewAggregateRule implements - RewriteRuleFactory { +public class MaterializedViewProjectAggregateRule extends AbstractMaterializedViewAggregateRule { public static final MaterializedViewProjectAggregateRule INSTANCE = new MaterializedViewProjectAggregateRule(); @@ -41,6 +38,6 @@ public List buildRules() { logicalProject(logicalAggregate(any())).thenApplyMulti(ctx -> { LogicalProject> root = ctx.root; return rewrite(root, ctx.cascadesContext); - }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_AGGREGATE, RulePromise.EXPLORE)); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_AGGREGATE)); } } 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..442a07d06a7e4b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectFilterJoinRule.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.RuleType; +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 { + + 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)); + } +} 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..283ea04ba43cb2 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 @@ -18,9 +18,7 @@ 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 org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -30,9 +28,9 @@ 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 { +public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoinRule { public static final MaterializedViewProjectJoinRule INSTANCE = new MaterializedViewProjectJoinRule(); @@ -42,6 +40,6 @@ public List buildRules() { logicalProject(logicalJoin(any(), any())).thenApplyMulti(ctx -> { LogicalProject> root = ctx.root; return rewrite(root, ctx.cascadesContext); - }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_JOIN, RulePromise.EXPLORE)); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_JOIN)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java index c5909822adbb18..c9624c176b9fd9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java @@ -18,14 +18,13 @@ package org.apache.doris.nereids.rules.exploration.mv; import org.apache.doris.nereids.rules.Rule; -import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; import java.util.List; /** * This is responsible for single table rewriting according to different pattern * */ -public class MaterializedViewScanRule extends AbstractMaterializedViewRule implements RewriteRuleFactory { +public class MaterializedViewScanRule extends AbstractMaterializedViewRule { @Override public List buildRules() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index 13e2a5bd469c6c..1d38e893dba481 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -282,7 +282,9 @@ private static class PredicateCollector extends DefaultPlanVisitor predicates) { // Just collect the filter in top plan, if meet other node except project and filter, return - if (!(plan instanceof LogicalProject) && !(plan instanceof LogicalFilter)) { + if (!(plan instanceof LogicalProject) + && !(plan instanceof LogicalFilter) + && !(plan instanceof LogicalAggregate)) { return null; } if (plan instanceof LogicalFilter) { @@ -396,7 +398,7 @@ public Boolean visit(Plan plan, Void context) { super.visit(aggregate, context); return true; } - if (plan instanceof LogicalProject) { + if (plan instanceof LogicalProject || plan instanceof LogicalFilter) { super.visit(plan, context); return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Any.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Any.java new file mode 100644 index 00000000000000..43d284bf67838c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Any.java @@ -0,0 +1,67 @@ +// 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.trees.expressions; + +import org.apache.doris.nereids.trees.TreeNode; +import org.apache.doris.nereids.trees.expressions.shape.LeafExpression; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This represents any expression, it means it equals any expression + */ +public class Any extends Expression implements LeafExpression { + + public static final Any INSTANCE = new Any(ImmutableList.of()); + + private Any(Expression... children) { + super(children); + } + + private Any(List children) { + super(children); + } + + @Override + public R accept(ExpressionVisitor visitor, C context) { + return visitor.visitAny(this, context); + } + + @Override + public boolean nullable() { + return false; + } + + @Override + public boolean equals(Object o) { + return true; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean deepEquals(TreeNode that) { + return true; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java index 61a589daba2781..a7e523dfdb549e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/AggregateFunction.java @@ -77,10 +77,6 @@ public boolean isDistinct() { return distinct; } - public Class getRollup() { - return null; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CouldRollUp.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CouldRollUp.java new file mode 100644 index 00000000000000..b3e1674d3ed6ab --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/CouldRollUp.java @@ -0,0 +1,32 @@ +// 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.trees.expressions.functions.agg; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.Function; + +/** + * Could roll up trait, if a function could roll up in aggregate, it will implement the interface + */ +public interface CouldRollUp { + + /** + * construct the roll up function with custom param + */ + Function constructRollUp(Expression param, Expression... varParams); +} 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 a8a3fdd033ddc4..afd1b39dbc12f3 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 @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; 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.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -36,7 +37,7 @@ /** count agg function. */ public class Count extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic { + implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic, CouldRollUp { public static final List SIGNATURES = ImmutableList.of( // count(*) @@ -142,4 +143,13 @@ public R accept(ExpressionVisitor visitor, C context) { public List getSignatures() { return SIGNATURES; } + + @Override + public Function constructRollUp(Expression param, Expression... varParams) { + if (this.isDistinct()) { + return new BitmapUnionCount(param); + } else { + return new Sum(param); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java index 19cd0190bb6e37..a8530bbc18092e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -34,7 +35,7 @@ /** max agg function. */ public class Max extends NullableAggregateFunction - implements UnaryExpression, CustomSignature, SupportWindowAnalytic { + implements UnaryExpression, CustomSignature, SupportWindowAnalytic, CouldRollUp { public Max(Expression child) { this(false, false, child); } @@ -80,4 +81,9 @@ public NullableAggregateFunction withAlwaysNullable(boolean alwaysNullable) { public R accept(ExpressionVisitor visitor, C context) { return visitor.visitMax(this, context); } + + @Override + public Function constructRollUp(Expression param, Expression... varParams) { + return new Max(this.distinct, this.alwaysNullable, param); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java index 72b2162eb51f6e..d40c32844e180e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.Function; import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -34,7 +35,7 @@ /** min agg function. */ public class Min extends NullableAggregateFunction - implements UnaryExpression, CustomSignature, SupportWindowAnalytic { + implements UnaryExpression, CustomSignature, SupportWindowAnalytic, CouldRollUp { public Min(Expression child) { this(false, false, child); @@ -81,4 +82,9 @@ public NullableAggregateFunction withAlwaysNullable(boolean alwaysNullable) { public R accept(ExpressionVisitor visitor, C context) { return visitor.visitMin(this, context); } + + @Override + public Function constructRollUp(Expression param, Expression... varParams) { + return new Min(this.distinct, this.alwaysNullable, param); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java index b99f836e09e8b2..34681a2611623f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; 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.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -45,7 +46,8 @@ * AggregateFunction 'sum'. This class is generated by GenerateFunction. */ public class Sum extends NullableAggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic { + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic, + CouldRollUp { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(BooleanType.INSTANCE), @@ -111,7 +113,7 @@ public List getSignatures() { } @Override - public Class getRollup() { - return Sum.class; + public Function constructRollUp(Expression param, Expression... varParams) { + return new Sum(this.distinct, param); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java index 8aee07182d1ef6..b53d22916a19ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.And; +import org.apache.doris.nereids.trees.expressions.Any; import org.apache.doris.nereids.trees.expressions.ArrayItemReference; import org.apache.doris.nereids.trees.expressions.AssertNumRowsElement; import org.apache.doris.nereids.trees.expressions.BinaryArithmetic; @@ -499,6 +500,10 @@ public R visitMatchPhrasePrefix(MatchPhrasePrefix matchPhrasePrefix, C context) return visitMatch(matchPhrasePrefix, context); } + public R visitAny(Any any, C context) { + return visit(any, context); + } + /* ******************************************************************************************** * Unbound expressions * ********************************************************************************************/ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/ExpressionLineageReplacer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/ExpressionLineageReplacer.java index 770ceafdfe643a..2a4b81a58285e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/ExpressionLineageReplacer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/ExpressionLineageReplacer.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer.ExpressionReplaceContext; +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; @@ -66,9 +67,28 @@ public static class ExpressionReplacer extends DefaultExpressionRewriter exprIdExpressionMap) { if (exprIdExpressionMap.containsKey(namedExpression.getExprId())) { - return super.visit(exprIdExpressionMap.get(namedExpression.getExprId()), exprIdExpressionMap); + return visit(exprIdExpressionMap.get(namedExpression.getExprId()), exprIdExpressionMap); } - return super.visitNamedExpression(namedExpression, exprIdExpressionMap); + return visit(namedExpression, exprIdExpressionMap); + } + + @Override + public Expression visit(Expression expr, Map exprIdExpressionMap) { + if (expr instanceof NamedExpression + && expr.arity() == 0 + && exprIdExpressionMap.containsKey(((NamedExpression) expr).getExprId())) { + expr = exprIdExpressionMap.get(((NamedExpression) expr).getExprId()); + } + List newChildren = new ArrayList<>(expr.arity()); + boolean hasNewChildren = false; + for (Expression child : expr.children()) { + Expression newChild = child.accept(this, exprIdExpressionMap); + if (newChild != child) { + hasNewChildren = true; + } + newChildren.add(newChild); + } + return hasNewChildren ? expr.withChildren(newChildren) : expr; } } 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/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index f6dba53996235a..2e402cd5c7aa38 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -122,7 +122,7 @@ public void getRelatedTableInfoTestWithoutGroupTest() { } @Test - public void getRelatedTableInfoTestWithAliasAndGroupTest() { + public void getRelatedTableInfoTestWithSubqueryTest() { PlanChecker.from(connectContext) .checkExplain("SELECT l.L_SHIPDATE AS ship_data_alias, o.O_ORDERDATE, count(*) " + "FROM " @@ -147,6 +147,36 @@ public void getRelatedTableInfoTestWithAliasAndGroupTest() { }); } + @Test + public void getRelatedTableInfoTestWithAliasAndGroupTest() { + PlanChecker.from(connectContext) + .checkExplain("SELECT t1.L_SHIPDATE, t2.O_ORDERDATE, t1.L_QUANTITY, t2.O_ORDERSTATUS, " + + "count(distinct case when t1.L_SUPPKEY > 0 then t2.O_ORDERSTATUS else null end) as cnt_1 " + + "from " + + " (select * from " + + " lineitem " + + " where L_SHIPDATE in ('2017-01-30')) t1 " + + "left join " + + " (select * from " + + " orders " + + " where O_ORDERDATE in ('2017-01-30')) t2 " + + "on t1.L_ORDERKEY = t2.O_ORDERKEY " + + "group by " + + "t1.L_SHIPDATE, " + + "t2.O_ORDERDATE, " + + "t1.L_QUANTITY, " + + "t2.O_ORDERSTATUS;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + Optional relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfo("L_SHIPDATE", rewrittenPlan); + checkRelatedTableInfo(relatedTableInfo, + "lineitem", + "L_SHIPDATE", + true); + }); + } + @Test public void getRelatedTableInfoTestWithoutPartitionTest() { PlanChecker.from(connectContext) diff --git a/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out new file mode 100644 index 00000000000000..b240c02f7eaaf2 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.out @@ -0,0 +1,123 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query13_0_before -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !query13_0_after -- +3 3 2023-12-11 43.20 43.20 43.20 1 0 + +-- !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 + +-- !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/agg_without_roll_up/aggregate_without_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out new file mode 100644 index 00000000000000..346814bc083bf7 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out @@ -0,0 +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 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/join/inner/inner_join.out b/regression-test/data/nereids_rules_p0/mv/join/inner/inner_join.out new file mode 100644 index 00000000000000..7ae2e05e523327 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/join/inner/inner_join.out @@ -0,0 +1,267 @@ +-- 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 -- +4 +4 +4 +4 +6 +6 +6 +6 + +-- !query1_4_after -- +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 -- + +-- !query10_0_after -- + diff --git a/regression-test/data/nereids_rules_p0/mv/join/left_outer/outer_join.out b/regression-test/data/nereids_rules_p0/mv/join/left_outer/outer_join.out new file mode 100644 index 00000000000000..94143e800ada97 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/join/left_outer/outer_join.out @@ -0,0 +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 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 + +-- !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 + +-- !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 + +-- !query4_0_before -- +4 +4 + +-- !query4_0_after -- +4 +4 + +-- !query5_0_before -- +4 +4 +4 +4 +4 +4 +6 +6 + +-- !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/agg_with_roll_up/aggregate_with_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy index a42fa119f75e16..d6d96acbb8881b 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_with_roll_up/aggregate_with_roll_up.groovy @@ -22,6 +22,8 @@ suite("aggregate_with_roll_up") { 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 @@ -29,18 +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) - 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" ) @@ -52,25 +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) - 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" ) @@ -82,33 +86,39 @@ 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" ) """ - waiteCreateTableFinished("lineitem") 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');""" + (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'); + """ - waiteCreateTableFinished("orders") 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'), - (2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'); + (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'); """ - waiteCreateTableFinished("partsupp") sql """ insert into partsupp values (2, 3, 9, 10.01, 'supply1'), @@ -134,20 +144,571 @@ 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 " + + 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})" + } + } + + // 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, " + + "sum(o_totalprice), " + + "max(o_totalprice), " + + "min(o_totalprice), " + + "count(*) " + + "from orders " + + "where o_orderdate = '2023-12-09' " + + "group by " + + "o_shippriority, " + + "o_comment " + // rewrite success but cbo not chose, 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""" + + + // 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 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_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 " + - "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 " + + "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 " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + - "group by lineitem.L_LINENUMBER" - // query - // fix later -// 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""" -} + "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_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""" + + // 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 (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/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index a2682cd86942e9..055f8d01804a65 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -22,6 +22,8 @@ suite("aggregate_without_roll_up") { 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 @@ -29,18 +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) - 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" ) @@ -52,25 +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) - 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" ) @@ -82,33 +86,39 @@ 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" ) """ - waiteCreateTableFinished("lineitem") 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');""" + (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'); + """ - waiteCreateTableFinished("orders") 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'), - (2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'); + (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'); """ - waiteCreateTableFinished("partsupp") sql """ insert into partsupp values (2, 3, 9, 10.01, 'supply1'), @@ -134,19 +144,625 @@ 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 " + + 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})" + } + } + + // 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""" + + + 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, " + + "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'" + + "group by " + + "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_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 " + // 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""" + + + 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, " + + "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' " + + "group by " + + "lineitem.L_ORDERKEY, " + + "orders.O_SHIPPRIORITY, " + + "orders.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, " + + "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 (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' " + + "group by " + + "lineitem.L_ORDERKEY, " + + "orders.O_SHIPPRIORITY, " + + "orders.O_COMMENT " + // 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 query1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, sum(O_TOTALPRICE) as sum_alias " + + 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 " - // fix later -// 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""" -} + 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/join/inner/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy new file mode 100644 index 00000000000000..e0c0452415dae7 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/join/inner/inner_join.groovy @@ -0,0 +1,428 @@ +// 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("inner_join") { + 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) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + 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) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp + """ + + 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 + ) + 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, 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, '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'), + (2, 3, 10, 11.01, 'supply2'); + """ + + 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})" + } + } + + // without filter + def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + 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""" + + + def mv1_1 = "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_1 = "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_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""" + + def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_2 = "select lineitem.L_LINENUMBER " + + "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 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 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") + order_qt_query1_3_after "${query1_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" + + 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 " + + "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY" + def query10_0 = "select orders.O_CUSTKEY " + + "from orders " + + "inner join lineitem on orders.O_ORDERKEY = lineitem.L_ORDERKEY " + + "WHERE lineitem.L_LINENUMBER > 10 AND orders.O_CUSTKEY = 5 AND " + + "orders.O_SHIPPRIORITY = 1" + order_qt_query10_0_before "${query10_0}" + check_not_match(mv10_0, query10_0, "mv10_0") + order_qt_query10_0_after "${query10_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv10_0""" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/join/inner_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/inner_join.groovy deleted file mode 100644 index 4b6945b1542831..00000000000000 --- a/regression-test/suites/nereids_rules_p0/mv/join/inner_join.groovy +++ /dev/null @@ -1,207 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -suite("inner_join") { - 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" - - 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) - 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) - DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - sql """ - drop table if exists partsupp - """ - - 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 - ) - DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) - DISTRIBUTED BY HASH(PS_PARTKEY) BUCKETS 3 - PROPERTIES ( - "replication_num" = "1" - ) - """ - - waiteCreateTableFinished("lineitem") - 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');""" - - waiteCreateTableFinished("orders") - 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'), - (2, 3, 'ok', 109.2, '2023-12-09', 'c','d',2, 'mm'); - """ - - waiteCreateTableFinished("partsupp") - sql """ - insert into partsupp values - (2, 3, 9, 10.01, 'supply1'), - (2, 3, 10, 11.01, 'supply2'); - """ - - 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})" - } - } - -// // select + from + inner join - def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query1_0 = "select lineitem.L_LINENUMBER " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - // fix later -// 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""" - - - def mv1_1 = "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_1 = "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" - // fix later -// 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""" - - - def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + - "from orders " + - "inner join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - def query1_2 = "select lineitem.L_LINENUMBER " + - "from lineitem " + - "inner join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " - // fix later -// 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""" - - // select + from + inner join + filter - def mv1_3 = "select lineitem.L_LINENUMBER, orders.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" - // fix later -// 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" - // fix later -// 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""" -} diff --git a/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy new file mode 100644 index 00000000000000..d7ec67189ac871 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/join/left_outer/outer_join.groovy @@ -0,0 +1,364 @@ +// 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("outer_join") { + 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) (PARTITION `day_2` VALUES LESS THAN ('2023-12-30')) + 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) (PARTITION `day_1` VALUES LESS THAN ('2023-12-30')) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists partsupp + """ + + 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 + ) + 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, 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, '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'), + (2, 3, 10, 11.01, 'supply2'); + """ + + 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})" + } + } + + // without filter + def mv1_0 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + 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""" + + + def mv1_1 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY, partsupp.PS_AVAILQTY " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + def query1_1 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "left join partsupp on lineitem.L_PARTKEY = partsupp.PS_PARTKEY " + + "and lineitem.L_SUPPKEY = partsupp.PS_SUPPKEY" + 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""" + + def mv1_2 = "select lineitem.L_LINENUMBER, orders.O_CUSTKEY " + + "from orders " + + "left join lineitem on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + def query1_2 = "select lineitem.L_LINENUMBER " + + "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 + 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, " + + "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}" + check_rewrite(mv1_3, query1_3, "mv1_3") + order_qt_query1_3_after "${query1_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_3""" + + + // 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 " + def query2_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "where lineitem.L_LINENUMBER > 10" + order_qt_query2_0_before "${query2_0}" + 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 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 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 " + + "left join orders on t1.L_ORDERKEY = orders.O_ORDERKEY " + def query2_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "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""" + + + // 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 query3_0 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left 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 " + + "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 = 'o') t2 " + + "on lineitem.L_ORDERKEY = t2.O_ORDERKEY " + def query3_2 = "select lineitem.L_LINENUMBER " + + "from lineitem " + + "left join orders on lineitem.L_ORDERKEY = orders.O_ORDERKEY " + + "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(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""" + + + // 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 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..e0144fe636993f --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy @@ -0,0 +1,176 @@ +// 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_def_sql = """ + 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 all_partition_sql = """ + 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 partition_sql = """ + 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 + where (l_shipdate>= '2023-10-18' and l_shipdate <= '2023-10-19') + group by + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + + sql """DROP MATERIALIZED VIEW IF EXISTS mv_10086""" + sql """DROP TABLE IF EXISTS mv_10086""" + sql""" + CREATE MATERIALIZED VIEW mv_10086 + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + ${mv_def_sql} + """ + + def job_name = getJobName(db, "mv_10086"); + waitingMTMVTaskFinished(job_name) + + explain { + sql("${all_partition_sql}") + contains "mv_10086" + } + explain { + sql("${partition_sql}") + contains "mv_10086" + } + // partition is invalid, so can not 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(5000) + // only can use valid partition + explain { + sql("${all_partition_sql}") + notContains "mv_10086" + } + explain { + sql("${partition_sql}") + contains "mv_10086" + } +}