diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java index 1dff2159ad5959..7fe120d4dc134c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/profile/SummaryProfile.java @@ -87,6 +87,8 @@ public class SummaryProfile { public static final String NEREIDS_LOCK_TABLE_TIME = "Nereids Lock Table Time"; public static final String NEREIDS_ANALYSIS_TIME = "Nereids Analysis Time"; public static final String NEREIDS_REWRITE_TIME = "Nereids Rewrite Time"; + + public static final String NEREIDS_COLLECT_TABLE_PARTITION_TIME = "Nereids Collect Table Partition Time"; public static final String NEREIDS_OPTIMIZE_TIME = "Nereids Optimize Time"; public static final String NEREIDS_TRANSLATE_TIME = "Nereids Translate Time"; public static final String NEREIDS_GARBAGE_COLLECT_TIME = "Nereids GarbageCollect Time"; @@ -198,6 +200,8 @@ public class SummaryProfile { private long parseSqlFinishTime = -1; private long nereidsLockTableFinishTime = -1; + + private long nereidsCollectTablePartitionFinishTime = -1; private long nereidsAnalysisFinishTime = -1; private long nereidsRewriteFinishTime = -1; private long nereidsOptimizeFinishTime = -1; @@ -318,6 +322,8 @@ private void updateExecutionSummaryProfile() { executionSummaryProfile.addInfoString(NEREIDS_LOCK_TABLE_TIME, getPrettyNereidsLockTableTime()); executionSummaryProfile.addInfoString(NEREIDS_ANALYSIS_TIME, getPrettyNereidsAnalysisTime()); executionSummaryProfile.addInfoString(NEREIDS_REWRITE_TIME, getPrettyNereidsRewriteTime()); + executionSummaryProfile.addInfoString(NEREIDS_COLLECT_TABLE_PARTITION_TIME, + getPrettyNereidsCollectTablePartitionTime()); executionSummaryProfile.addInfoString(NEREIDS_OPTIMIZE_TIME, getPrettyNereidsOptimizeTime()); executionSummaryProfile.addInfoString(NEREIDS_TRANSLATE_TIME, getPrettyNereidsTranslateTime()); executionSummaryProfile.addInfoString(NEREIDS_GARBAGE_COLLECT_TIME, getPrettyNereidsGarbageCollectionTime()); @@ -406,6 +412,10 @@ public void setNereidsLockTableFinishTime() { this.nereidsLockTableFinishTime = TimeUtils.getStartTimeMs(); } + public void setNereidsCollectTablePartitionFinishTime() { + this.nereidsCollectTablePartitionFinishTime = TimeUtils.getStartTimeMs(); + } + public void setNereidsAnalysisTime() { this.nereidsAnalysisFinishTime = TimeUtils.getStartTimeMs(); } @@ -663,8 +673,13 @@ public String getPrettyNereidsRewriteTime() { return getPrettyTime(nereidsRewriteFinishTime, nereidsAnalysisFinishTime, TUnit.TIME_MS); } + + public String getPrettyNereidsCollectTablePartitionTime() { + return getPrettyTime(nereidsCollectTablePartitionFinishTime, nereidsRewriteFinishTime, TUnit.TIME_MS); + } + public String getPrettyNereidsOptimizeTime() { - return getPrettyTime(nereidsOptimizeFinishTime, nereidsRewriteFinishTime, TUnit.TIME_MS); + return getPrettyTime(nereidsOptimizeFinishTime, nereidsCollectTablePartitionFinishTime, TUnit.TIME_MS); } public String getPrettyNereidsTranslateTime() { 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 7858aa952777c9..bd9244af61e71f 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 @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.MTMV; +import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.Table; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.DdlException; @@ -27,6 +28,7 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.job.extensions.mtmv.MTMVTask; import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; +import org.apache.doris.nereids.rules.exploration.mv.PartitionCompensator; import org.apache.doris.nereids.trees.plans.commands.info.CancelMTMVTaskInfo; import org.apache.doris.nereids.trees.plans.commands.info.PauseMTMVInfo; import org.apache.doris.nereids.trees.plans.commands.info.RefreshMTMVInfo; @@ -43,6 +45,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -82,13 +85,21 @@ public Set getAvailableMTMVs(List tableInfos, ConnectContex boolean forceConsistent, BiPredicate predicate) { Set res = Sets.newLinkedHashSet(); Set mvInfos = getMTMVInfos(tableInfos); + Map, Set> queryUsedPartitions = PartitionCompensator.getQueryUsedPartitions( + ctx.getStatementContext()); + for (BaseTableInfo tableInfo : mvInfos) { try { MTMV mtmv = (MTMV) MTMVUtil.getTable(tableInfo); if (predicate.test(ctx, mtmv)) { continue; } - if (isMVPartitionValid(mtmv, ctx, forceConsistent)) { + if (!mtmv.isUseForRewrite()) { + continue; + } + BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); + if (isMVPartitionValid(mtmv, ctx, forceConsistent, + relatedTableInfo == null ? null : queryUsedPartitions.get(relatedTableInfo.toList()))) { res.add(mtmv); } } catch (Exception e) { @@ -117,10 +128,15 @@ public Set getAllMTMVs(List tableInfos) { } @VisibleForTesting - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, + Set relatedPartitions) { long currentTimeMillis = System.currentTimeMillis(); - return !CollectionUtils - .isEmpty(MTMVRewriteUtil.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMillis, forceConsistent)); + Collection mtmvCanRewritePartitions = MTMVRewriteUtil.getMTMVCanRewritePartitions( + mtmv, ctx, currentTimeMillis, forceConsistent, relatedPartitions); + // MTMVRewriteUtil.getMTMVCanRewritePartitions is time-consuming behavior, So record for used later + ctx.getStatementContext().getMvCanRewritePartitionsMap().putIfAbsent( + new BaseTableInfo(mtmv), mtmvCanRewritePartitions); + return !CollectionUtils.isEmpty(mtmvCanRewritePartitions); } private Set getMTMVInfos(List tableInfos) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java index ff1b3263d3409e..afaad55a34bbc4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java @@ -25,12 +25,16 @@ import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import java.util.Collection; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; public class MTMVRewriteUtil { private static final Logger LOG = LogManager.getLogger(MTMVRewriteUtil.class); @@ -43,7 +47,8 @@ public class MTMVRewriteUtil { * @return */ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, ConnectContext ctx, - long currentTimeMills, boolean forceConsistent) { + long currentTimeMills, boolean forceConsistent, + Set relatedPartitions) { List res = Lists.newArrayList(); Collection allPartitions = mtmv.getPartitions(); MTMVRelation mtmvRelation = mtmv.getRelation(); @@ -55,6 +60,11 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne if (mtmvStatus.getState() != MTMVState.NORMAL || mtmvStatus.getRefreshState() == MTMVRefreshState.INIT) { return res; } + // if relatedPartitions is empty but not null, which means query no partitions + if (relatedPartitions != null && relatedPartitions.size() == 0) { + return res; + } + Set mtmvNeedComparePartitions = null; MTMVRefreshContext refreshContext = null; // check gracePeriod long gracePeriodMills = mtmv.getGracePeriod(); @@ -73,6 +83,14 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne return res; } } + if (mtmvNeedComparePartitions == null) { + mtmvNeedComparePartitions = getMtmvPartitionsByRelatedPartitions(mtmv, refreshContext, + relatedPartitions); + } + // if the partition which query not used, should not compare partition version + if (!mtmvNeedComparePartitions.contains(partition.getName())) { + continue; + } try { if (MTMVPartitionUtil.isMTMVPartitionSync(refreshContext, partition.getName(), mtmvRelation.getBaseTablesOneLevel(), @@ -86,4 +104,29 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne } return res; } + + private static Set getMtmvPartitionsByRelatedPartitions(MTMV mtmv, MTMVRefreshContext refreshContext, + Set relatedPartitions) { + // if relatedPartitions is null, which means QueryPartitionCollector visitLogicalCatalogRelation can not + // get query used partitions, should get all mtmv partitions + if (relatedPartitions == null) { + return mtmv.getPartitionNames(); + } + Set res = Sets.newHashSet(); + Map relatedToMv = getRelatedToMv(refreshContext.getPartitionMappings()); + for (String relatedPartition : relatedPartitions) { + res.add(relatedToMv.get(relatedPartition)); + } + return res; + } + + private static Map getRelatedToMv(Map> mvToRelated) { + Map res = Maps.newHashMap(); + for (Entry> entry : mvToRelated.entrySet()) { + for (String relatedPartition : entry.getValue()) { + res.put(relatedPartition, entry.getKey()); + } + } + return res; + } } 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 1486f03e269b13..d431fc545396b7 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 @@ -24,6 +24,7 @@ import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; import org.apache.doris.nereids.jobs.executor.TableCollectAndHookInitializer; +import org.apache.doris.nereids.jobs.executor.TablePartitionCollector; import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob; import org.apache.doris.nereids.jobs.rewrite.RewriteTopDownJob; import org.apache.doris.nereids.jobs.rewrite.RootPlanTreeRewriteJob.RootRewriteJobContext; @@ -228,6 +229,10 @@ public TableCollectAndHookInitializer newTableCollector() { return new TableCollectAndHookInitializer(this); } + public TablePartitionCollector newTablePartitionCollector() { + return new TablePartitionCollector(this); + } + public Analyzer newAnalyzer() { return new Analyzer(this); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index 89a9d220be34e9..9e9765ec1398f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -344,7 +344,22 @@ protected void collectAndLockTable(boolean showPlanProcess) { } } - private void analyze(boolean showPlanProcess) { + protected void collectTableUsedPartitions(boolean showPlanProcess) { + if (LOG.isDebugEnabled()) { + LOG.debug("Start to collect table used partition"); + } + keepOrShowPlanProcess(showPlanProcess, () -> cascadesContext.newTablePartitionCollector().execute()); + NereidsTracer.logImportantTime("EndCollectTablePartitions"); + if (LOG.isDebugEnabled()) { + LOG.debug("Start to collect table used partition"); + } + if (statementContext.getConnectContext().getExecutor() != null) { + statementContext.getConnectContext().getExecutor().getSummaryProfile() + .setNereidsCollectTablePartitionFinishTime(); + } + } + + protected void analyze(boolean showPlanProcess) { if (LOG.isDebugEnabled()) { LOG.debug("Start analyze plan"); } @@ -376,6 +391,10 @@ private void rewrite(boolean showPlanProcess) { if (statementContext.getConnectContext().getExecutor() != null) { statementContext.getConnectContext().getExecutor().getSummaryProfile().setNereidsRewriteTime(); } + // collect partitions table used, this is for query rewrite by materialized view + // this is needed before init hook + collectTableUsedPartitions(showPlanProcess); + cascadesContext.getStatementContext().getPlannerHooks().forEach(hook -> hook.afterRewrite(this)); } // DependsRules: EnsureProjectOnTopJoin.class diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerHook.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerHook.java index 18d71b539a94b3..76a68f0b22c763 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerHook.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerHook.java @@ -35,4 +35,16 @@ default void beforeAnalyze(NereidsPlanner planner) { */ default void afterAnalyze(NereidsPlanner planner) { } + + /** + * the hook before rewrite + */ + default void beforeRewrite(NereidsPlanner planner) { + } + + /** + * the hook after rewrite + */ + default void afterRewrite(NereidsPlanner planner) { + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index de7f041a4629a0..c2a0cace6bb256 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Partition; import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.View; import org.apache.doris.catalog.constraint.TableIdentifier; @@ -29,6 +30,7 @@ import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.datasource.mvcc.MvccTable; import org.apache.doris.datasource.mvcc.MvccTableInfo; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.hint.Hint; import org.apache.doris.nereids.memo.Group; @@ -60,6 +62,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.base.Throwables; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.Maps; import com.google.common.collect.Multimap; @@ -106,6 +109,7 @@ public enum TableFrom { private ConnectContext connectContext; private final Stopwatch stopwatch = Stopwatch.createUnstarted(); + private final Stopwatch materializedViewStopwatch = Stopwatch.createUnstarted(); @GuardedBy("this") private final Map> contextCacheMap = Maps.newLinkedHashMap(); @@ -175,7 +179,14 @@ public enum TableFrom { // tables in this query directly private final Map, TableIf> tables = Maps.newHashMap(); - // tables maybe used by mtmv rewritten in this query + // tables maybe used by mtmv rewritten in this query, + // this contains mvs which use table in tables and the tables in mvs + // such as + // mv1 use t1, t2. + // mv2 use mv1, t3, t4. + // mv3 use t3, t4, t5 + // if query is: select * from t2 join t5 + // mtmvRelatedTables is mv1, mv2, mv3, t1, t2, t3, t4, t5 private final Map, TableIf> mtmvRelatedTables = Maps.newHashMap(); // insert into target tables private final Map, TableIf> insertTargetTables = Maps.newHashMap(); @@ -211,6 +222,16 @@ public enum TableFrom { private boolean privChecked; + // if greater than 0 means the duration has used + private long materializedViewRewriteDuration = 0L; + + // Record used table and it's used partitions + private final Multimap, Pair>> tableUsedPartitionNameMap = + HashMultimap.create(); + + // Record mtmv and valid partitions map because this is time-consuming behavior + private final Map> mvCanRewritePartitionsMap = new HashMap<>(); + public StatementContext() { this(ConnectContext.get(), null, 0); } @@ -339,6 +360,26 @@ public Stopwatch getStopwatch() { return stopwatch; } + public Stopwatch getMaterializedViewStopwatch() { + return materializedViewStopwatch; + } + + public long getMaterializedViewRewriteDuration() { + return materializedViewRewriteDuration; + } + + public void addMaterializedViewRewriteDuration(long millisecond) { + materializedViewRewriteDuration += millisecond; + } + + public Multimap, Pair>> getTableUsedPartitionNameMap() { + return tableUsedPartitionNameMap; + } + + public Map> getMvCanRewritePartitionsMap() { + return mvCanRewritePartitionsMap; + } + public void setMaxNAryInnerJoin(int maxNAryInnerJoin) { if (maxNAryInnerJoin > this.maxNAryInnerJoin) { this.maxNAryInnerJoin = maxNAryInnerJoin; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TablePartitionCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TablePartitionCollector.java new file mode 100644 index 00000000000000..e67b94d1314e89 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/TablePartitionCollector.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.jobs.executor; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.QueryPartitionCollector; + +import java.util.List; + +/** + * Collect partitions which query used, this is useful for optimizing get available mvs, + * should collect after RBO + */ +public class TablePartitionCollector extends AbstractBatchJobExecutor { + public TablePartitionCollector(CascadesContext cascadesContext) { + super(cascadesContext); + } + + @Override + public List getJobs() { + return buildCollectorJobs(); + } + + private static List buildCollectorJobs() { + return jobs( + custom(RuleType.COLLECT_PARTITIONS, QueryPartitionCollector::new) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/StructInfoMap.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/StructInfoMap.java index c39be5b569089f..36079fd3763475 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/StructInfoMap.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/StructInfoMap.java @@ -25,6 +25,8 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; 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.BitSet; @@ -42,6 +44,8 @@ * Representation for group in cascades optimizer. */ public class StructInfoMap { + + public static final Logger LOG = LogManager.getLogger(StructInfoMap.class); private final Map>> groupExpressionMap = new HashMap<>(); private final Map infoMap = new HashMap<>(); private long refreshVersion = 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index e401330a65b725..7db8ac570d6447 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -342,6 +342,7 @@ public enum RuleType { LEADING_JOIN(RuleTypeClass.REWRITE), REWRITE_SENTINEL(RuleTypeClass.REWRITE), + COLLECT_PARTITIONS(RuleTypeClass.REWRITE), // topn opts DEFER_MATERIALIZE_TOP_N_RESULT(RuleTypeClass.REWRITE), 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 e8e46b68f3da38..b079f0d12352bc 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 @@ -18,17 +18,13 @@ package org.apache.doris.nereids.rules.exploration.mv; import org.apache.doris.catalog.MTMV; -import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.PartitionInfo; -import org.apache.doris.catalog.PartitionType; import org.apache.doris.catalog.constraint.TableIdentifier; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Id; import org.apache.doris.common.Pair; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.mtmv.MTMVPartitionInfo; -import org.apache.doris.mtmv.MTMVRewriteUtil; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.rules.exploration.ExplorationRuleFactory; @@ -56,7 +52,6 @@ import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.types.VariantType; @@ -66,7 +61,6 @@ import org.apache.doris.statistics.Statistics; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -75,15 +69,13 @@ import java.util.ArrayList; import java.util.BitSet; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; /** @@ -100,7 +92,8 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac JoinType.LEFT_SEMI_JOIN, JoinType.RIGHT_SEMI_JOIN, JoinType.LEFT_ANTI_JOIN, - JoinType.RIGHT_ANTI_JOIN); + JoinType.RIGHT_ANTI_JOIN, + JoinType.NULL_AWARE_LEFT_ANTI_JOIN); /** * The abstract template method for query rewrite, it contains the main logic, try to rewrite query by @@ -109,11 +102,22 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac */ public List rewrite(Plan queryPlan, CascadesContext cascadesContext) { List rewrittenPlans = new ArrayList<>(); + SessionVariable sessionVariable = cascadesContext.getConnectContext().getSessionVariable(); // if available materialization list is empty, bail out + StatementContext statementContext = cascadesContext.getStatementContext(); if (cascadesContext.getMaterializationContexts().isEmpty()) { return rewrittenPlans; } + if (statementContext.getMaterializedViewRewriteDuration() + > sessionVariable.materializedViewRewriteDurationThresholdMs) { + LOG.warn("materialized view rewrite duration is exceeded, the query queryId is {}", + cascadesContext.getConnectContext().getQueryIdentifier()); + MaterializationContext.makeFailWithDurationExceeded(queryPlan, cascadesContext.getMaterializationContexts(), + statementContext.getMaterializedViewRewriteDuration()); + return rewrittenPlans; + } for (MaterializationContext context : cascadesContext.getMaterializationContexts()) { + statementContext.getMaterializedViewStopwatch().reset().start(); if (checkIfRewritten(queryPlan, context)) { continue; } @@ -127,15 +131,31 @@ public List rewrite(Plan queryPlan, CascadesContext cascadesContext) { if (queryStructInfos.isEmpty()) { continue; } + statementContext.addMaterializedViewRewriteDuration( + statementContext.getMaterializedViewStopwatch().elapsed(TimeUnit.MILLISECONDS)); for (StructInfo queryStructInfo : queryStructInfos) { + statementContext.getMaterializedViewStopwatch().reset().start(); + if (statementContext.getMaterializedViewRewriteDuration() + > sessionVariable.materializedViewRewriteDurationThresholdMs) { + statementContext.getMaterializedViewStopwatch().stop(); + LOG.warn("materialized view rewrite duration is exceeded, the queryId is {}", + cascadesContext.getConnectContext().getQueryIdentifier()); + MaterializationContext.makeFailWithDurationExceeded(queryStructInfo.getOriginalPlan(), + cascadesContext.getMaterializationContexts(), + statementContext.getMaterializedViewRewriteDuration()); + return rewrittenPlans; + } try { - if (rewrittenPlans.size() < cascadesContext.getConnectContext() - .getSessionVariable().getMaterializedViewRewriteSuccessCandidateNum()) { + if (rewrittenPlans.size() < sessionVariable.getMaterializedViewRewriteSuccessCandidateNum()) { rewrittenPlans.addAll(doRewrite(queryStructInfo, cascadesContext, context)); } } catch (Exception exception) { + LOG.warn("Materialized view rule exec fail", exception); context.recordFailReason(queryStructInfo, "Materialized view rule exec fail", exception::toString); + } finally { + statementContext.addMaterializedViewRewriteDuration( + statementContext.getMaterializedViewStopwatch().elapsed(TimeUnit.MILLISECONDS)); } } } @@ -266,43 +286,70 @@ protected List doRewrite(StructInfo queryStructInfo, CascadesContext casca continue; } Pair>, Map>> invalidPartitions; - if (materializationContext instanceof AsyncMaterializationContext) { + if (PartitionCompensator.needUnionRewrite(materializationContext) + && sessionVariable.isEnableMaterializedViewUnionRewrite()) { + MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); + BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); + Map, Set> queryUsedPartitions = PartitionCompensator.getQueryUsedPartitions( + cascadesContext.getConnectContext().getStatementContext()); + Set relateTableUsedPartitions = queryUsedPartitions.get(relatedTableInfo.toList()); + if (relateTableUsedPartitions == null) { + materializationContext.recordFailReason(queryStructInfo, + String.format("queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", + relatedTableInfo.toList(), + cascadesContext.getConnectContext().getQueryIdentifier()), + () -> String.format( + "queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", + relatedTableInfo.toList(), + cascadesContext.getConnectContext().getQueryIdentifier())); + LOG.warn(String.format( + "queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", + relatedTableInfo.toList(), cascadesContext.getConnectContext().getQueryIdentifier())); + return rewriteResults; + } + if (relateTableUsedPartitions.isEmpty()) { + materializationContext.recordFailReason(queryStructInfo, + String.format("queryUsedPartition is empty, table is %s, queryId is %s", + relatedTableInfo.toList(), + cascadesContext.getConnectContext().getQueryIdentifier()), + () -> String.format("queryUsedPartition is empty, table is %s, queryId is %s", + relatedTableInfo.toList(), + cascadesContext.getConnectContext().getQueryIdentifier())); + LOG.debug(String.format("queryUsedPartition is empty, table is %s, queryId is %s", + relatedTableInfo.toList(), cascadesContext.getConnectContext().getQueryIdentifier())); + // no need to rewrite by current mv, becaus + return rewriteResults; + } try { - invalidPartitions = calcInvalidPartitions(queryPlan, rewrittenPlan, - (AsyncMaterializationContext) materializationContext, cascadesContext); + invalidPartitions = calcInvalidPartitions(relateTableUsedPartitions, rewrittenPlan, + cascadesContext, (AsyncMaterializationContext) materializationContext); } catch (AnalysisException e) { materializationContext.recordFailReason(queryStructInfo, "Calc invalid partitions fail", () -> String.format("Calc invalid partitions fail, mv partition names are %s", - ((AsyncMaterializationContext) materializationContext).getMtmv().getPartitions())); + mtmv.getPartitions())); LOG.warn("Calc invalid partitions fail", e); continue; } if (invalidPartitions == null) { - // if mv can not offer any partition for query, query rewrite bail out to avoid cycle run materializationContext.recordFailReason(queryStructInfo, "mv can not offer any partition for query", - () -> String.format("mv partition info %s", - ((AsyncMaterializationContext) materializationContext).getMtmv() - .getMvPartitionInfo())); + () -> String.format("mv partition info %s", mtmv.getMvPartitionInfo())); + // if mv can not offer any partition for query, query rewrite bail out to avoid cycle run return rewriteResults; } - boolean partitionNeedUnion = needUnionRewrite(invalidPartitions, cascadesContext); - boolean canUnionRewrite = canUnionRewrite(queryPlan, - ((AsyncMaterializationContext) materializationContext).getMtmv(), - cascadesContext); + boolean partitionNeedUnion = PartitionCompensator.needUnionRewrite(invalidPartitions, cascadesContext); + boolean canUnionRewrite = canUnionRewrite(queryPlan, mtmv, cascadesContext); if (partitionNeedUnion && !canUnionRewrite) { materializationContext.recordFailReason(queryStructInfo, "need compensate union all, but can not, because the query structInfo", () -> String.format("mv partition info is %s, and the query plan is %s", - ((AsyncMaterializationContext) materializationContext).getMtmv() - .getMvPartitionInfo(), queryPlan.treeString())); + mtmv.getMvPartitionInfo(), queryPlan.treeString())); return rewriteResults; } final Pair>, Map>> finalInvalidPartitions = invalidPartitions; if (partitionNeedUnion) { - MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); Pair planAndNeedAddFilterPair = StructInfo.addFilterOnTableScan(queryPlan, invalidPartitions.value(), mtmv.getMvPartitionInfo().getRelatedCol(), cascadesContext); @@ -384,13 +431,6 @@ private static void trySetStatistics(MaterializationContext context, CascadesCon } } - protected boolean needUnionRewrite( - Pair>, Map>> invalidPartitions, - CascadesContext cascadesContext) { - return invalidPartitions != null - && (!invalidPartitions.key().isEmpty() || !invalidPartitions.value().isEmpty()); - } - /** * Not all query after rewritten successfully can compensate union all * Such as: @@ -438,98 +478,13 @@ protected boolean isOutputValid(Plan sourcePlan, Plan rewrittenPlan) { * @return the key in pair is mvNeedRemovePartitionNameSet, the value in pair is baseTableNeedUnionPartitionNameSet */ protected Pair>, Map>> calcInvalidPartitions( - Plan queryPlan, Plan rewrittenPlan, - AsyncMaterializationContext materializationContext, CascadesContext cascadesContext) + Set queryUsedPartition, + Plan rewrittenPlan, + CascadesContext cascadesContext, + AsyncMaterializationContext materializationContext) throws AnalysisException { - Set mvNeedRemovePartitionNameSet = new HashSet<>(); - Set baseTableNeedUnionPartitionNameSet = new HashSet<>(); - // 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 Pair.of(ImmutableMap.of(), ImmutableMap.of()); - } - MTMVPartitionInfo mvCustomPartitionInfo = mtmv.getMvPartitionInfo(); - BaseTableInfo relatedPartitionTable = mvCustomPartitionInfo.getRelatedTableInfo(); - if (relatedPartitionTable == null) { - return Pair.of(ImmutableMap.of(), ImmutableMap.of()); - } - // Collect the mv related base table partitions which query used - Map> queryUsedBaseTablePartitions = new LinkedHashMap<>(); - queryUsedBaseTablePartitions.put(relatedPartitionTable, new HashSet<>()); - queryPlan.accept(new StructInfo.QueryScanPartitionsCollector(), queryUsedBaseTablePartitions); - // Bail out, not check invalid partition if not olap scan, support later - if (queryUsedBaseTablePartitions.isEmpty()) { - return Pair.of(ImmutableMap.of(), ImmutableMap.of()); - } - Set queryUsedBaseTablePartitionNameSet = queryUsedBaseTablePartitions.get(relatedPartitionTable) - .stream() - .map(Partition::getName) - .collect(Collectors.toSet()); - - Collection mvValidPartitions = MTMVRewriteUtil.getMTMVCanRewritePartitions(mtmv, - cascadesContext.getConnectContext(), System.currentTimeMillis(), false); - Set mvValidPartitionNameSet = new HashSet<>(); - Set mvValidBaseTablePartitionNameSet = new HashSet<>(); - Set mvValidHasDataRelatedBaseTableNameSet = new HashSet<>(); - Pair>, Map> partitionMapping = mtmv.calculateDoublyPartitionMappings(); - for (Partition mvValidPartition : mvValidPartitions) { - mvValidPartitionNameSet.add(mvValidPartition.getName()); - Set relatedBaseTablePartitions = partitionMapping.key().get(mvValidPartition.getName()); - if (relatedBaseTablePartitions != null) { - mvValidBaseTablePartitionNameSet.addAll(relatedBaseTablePartitions); - } - if (!mtmv.selectNonEmptyPartitionIds(ImmutableList.of(mvValidPartition.getId())).isEmpty()) { - if (relatedBaseTablePartitions != null) { - mvValidHasDataRelatedBaseTableNameSet.addAll(relatedBaseTablePartitions); - } - } - } - if (Sets.intersection(mvValidHasDataRelatedBaseTableNameSet, queryUsedBaseTablePartitionNameSet).isEmpty()) { - // if mv can not offer any partition for query, query rewrite bail out - return null; - } - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("calcInvalidPartitions mv name is %s,\n mvValidBaseTablePartitionNameSet is %s,\n " - + "mvValidHasDataRelatedBaseTableNameSet is %s,\n" - + "queryUsedBaseTablePartitionNameSet is %s,\n " - + "partitionMapping is %s \n, sql hash is %s", - materializationContext.generateMaterializationIdentifier(), - mvValidBaseTablePartitionNameSet, - mvValidHasDataRelatedBaseTableNameSet, - queryUsedBaseTablePartitionNameSet, - partitionMapping, cascadesContext.getConnectContext().getSqlHash())); - } - // Check when mv partition relates base table partition data change or delete partition - Set rewrittenPlanUsePartitionNameSet = new HashSet<>(); - List mvOlapScanList = rewrittenPlan.collectToList(node -> - node instanceof LogicalOlapScan - && Objects.equals(((CatalogRelation) node).getTable().getName(), mtmv.getName())); - for (Object olapScanObj : mvOlapScanList) { - LogicalOlapScan olapScan = (LogicalOlapScan) olapScanObj; - olapScan.getSelectedPartitionIds().forEach(id -> - rewrittenPlanUsePartitionNameSet.add(olapScan.getTable().getPartition(id).getName())); - } - // If rewritten plan use but not in mv valid partition name set, need remove in mv and base table union - Sets.difference(rewrittenPlanUsePartitionNameSet, mvValidPartitionNameSet) - .copyInto(mvNeedRemovePartitionNameSet); - for (String partitionName : mvNeedRemovePartitionNameSet) { - baseTableNeedUnionPartitionNameSet.addAll(partitionMapping.key().get(partitionName)); - } - // If related base table create partitions or mv is created with ttl, need base table union - Sets.difference(queryUsedBaseTablePartitionNameSet, mvValidBaseTablePartitionNameSet) - .copyInto(baseTableNeedUnionPartitionNameSet); - // Construct result map - Map> mvPartitionNeedRemoveNameMap = new HashMap<>(); - if (!mvNeedRemovePartitionNameSet.isEmpty()) { - mvPartitionNeedRemoveNameMap.put(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet); - } - Map> baseTablePartitionNeedUnionNameMap = new HashMap<>(); - if (!baseTableNeedUnionPartitionNameSet.isEmpty()) { - baseTablePartitionNeedUnionNameMap.put(relatedPartitionTable, baseTableNeedUnionPartitionNameSet); - } - return Pair.of(mvPartitionNeedRemoveNameMap, baseTablePartitionNeedUnionNameMap); + return PartitionCompensator.calcInvalidPartitions(queryUsedPartition, rewrittenPlan, + materializationContext, cascadesContext); } /** @@ -900,6 +855,18 @@ protected boolean checkIfRewritten(Plan plan, MaterializationContext context) { // check mv plan is valid or not, this can use cache for performance private boolean isMaterializationValid(Plan queryPlan, CascadesContext cascadesContext, MaterializationContext context) { + if (!context.getStructInfo().isValid()) { + context.recordFailReason(context.getStructInfo(), + "View original struct info is invalid", () -> String.format("view plan is %s", + context.getStructInfo().getOriginalPlan().treeString())); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("View struct info is invalid, mv identifier is %s, query plan is %s," + + "view plan is %s", + context.generateMaterializationIdentifier(), queryPlan.treeString(), + context.getStructInfo().getTopPlan().treeString())); + } + return false; + } long materializationId = context.generateMaterializationIdentifier().hashCode(); Boolean cachedCheckResult = cascadesContext.getMemo().materializationHasChecked(this.getClass(), materializationId); @@ -936,18 +903,6 @@ private boolean isMaterializationValid(Plan queryPlan, CascadesContext cascadesC } return false; } - if (!context.getStructInfo().isValid()) { - context.recordFailReason(context.getStructInfo(), - "View original struct info is invalid", () -> String.format("view plan is %s", - context.getStructInfo().getOriginalPlan().treeString())); - if (LOG.isDebugEnabled()) { - LOG.debug(String.format("View struct info is invalid, mv identifier is %s, query plan is %s," - + "view plan is %s", - context.generateMaterializationIdentifier(), queryPlan.treeString(), - context.getStructInfo().getTopPlan().treeString())); - } - return false; - } return true; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java index ff1a5bdd09610b..186ac0bb447dd7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java @@ -406,13 +406,20 @@ private Map getQueryToViewNodeIdMap() { } private Map constructQueryToViewJoinMapWithExpr() { - Map viewExprToEdge = getViewJoinEdges().stream() - .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second)); - Map queryExprToEdge = getQueryJoinEdges().stream() - .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second)); - + Map viewExprToEdge = new HashMap<>(); + List viewJoinEdges = getViewJoinEdges(); + for (JoinEdge viewJoin : viewJoinEdges) { + for (Expression expression : viewJoin.getExpressions()) { + viewExprToEdge.put(expression, viewJoin); + } + } + Map queryExprToEdge = new HashMap<>(); + List queryJoinEdges = getQueryJoinEdges(); + for (JoinEdge queryJoin : queryJoinEdges) { + for (Expression expression : queryJoin.getExpressions()) { + queryExprToEdge.put(expression, queryJoin); + } + } HashMap edgeMap = new HashMap<>(); for (Entry entry : queryExprToEdge.entrySet()) { if (edgeMap.containsKey(entry.getValue())) { @@ -444,15 +451,19 @@ private Map constructQueryToViewJoinMapWithExpr() { // +--LogicalOlapScan private Map constructQueryToViewFilterMapWithExpr() { Multimap viewExprToEdge = HashMultimap.create(); - getViewFilterEdges().stream() - .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .forEach(pair -> viewExprToEdge.put(pair.key(), pair.value())); - + List viewFilterEdges = getViewFilterEdges(); + for (FilterEdge viewEdge : viewFilterEdges) { + for (Expression expression : viewEdge.getExpressions()) { + viewExprToEdge.put(expression, viewEdge); + } + } Multimap queryExprToEdge = HashMultimap.create(); - getQueryFilterEdges().stream() - .flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e))) - .forEach(pair -> queryExprToEdge.put(pair.key(), pair.value())); - + List queryFilterEdges = getQueryFilterEdges(); + for (FilterEdge queryEdge : queryFilterEdges) { + for (Expression expression : queryEdge.getExpressions()) { + queryExprToEdge.put(expression, queryEdge); + } + } HashMap queryToViewEdgeMap = new HashMap<>(); for (Entry> entry : queryExprToEdge.asMap().entrySet()) { Expression queryExprViewBased = null; 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 76651aa1169da8..205350e3b37202 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 @@ -49,7 +49,7 @@ public class InitMaterializationContextHook implements PlannerHook { public static final InitMaterializationContextHook INSTANCE = new InitMaterializationContextHook(); @Override - public void afterAnalyze(NereidsPlanner planner) { + public void afterRewrite(NereidsPlanner planner) { initMaterializationContext(planner.getCascadesContext()); } @@ -99,13 +99,13 @@ private List createAsyncMaterializationContext(CascadesC try { availableMTMVs = getAvailableMTMVs(usedTables, cascadesContext); } catch (Exception e) { - LOG.warn(String.format("MaterializationContext getAvailableMTMVs generate fail, current queryId is %s", - cascadesContext.getConnectContext().getQueryIdentifier()), e); + LOG.warn(String.format("MaterializationContext getAvailableMTMVs generate fail, current sqlHash is %s", + cascadesContext.getConnectContext().getSqlHash()), e); return ImmutableList.of(); } if (CollectionUtils.isEmpty(availableMTMVs)) { - LOG.debug("Enable materialized view rewrite but availableMTMVs is empty, current queryId " - + "is {}", cascadesContext.getConnectContext().getQueryIdentifier()); + LOG.debug("Enable materialized view rewrite but availableMTMVs is empty, current sqlHash " + + "is {}", cascadesContext.getConnectContext().getSqlHash()); return ImmutableList.of(); } List asyncMaterializationContext = new ArrayList<>(); @@ -113,13 +113,6 @@ private List createAsyncMaterializationContext(CascadesC MTMVCache mtmvCache = null; try { mtmvCache = materializedView.getOrGenerateCache(cascadesContext.getConnectContext()); - // If mv property use_for_rewrite is set false, should not partition in - // query rewrite by materialized view - if (!materializedView.isUseForRewrite()) { - LOG.debug("mv doesn't part in query rewrite process because " - + "use_for_rewrite is false, mv is {}", materializedView.getName()); - continue; - } if (mtmvCache == null) { continue; } 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 38eba2ac3406ff..46acd9861a4d96 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 @@ -24,6 +24,7 @@ import org.apache.doris.common.Id; import org.apache.doris.common.Pair; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.memo.GroupId; import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping; import org.apache.doris.nereids.rules.exploration.mv.mapping.RelationMapping; @@ -326,7 +327,7 @@ public boolean isSuccess() { } /** - * Record fail reason when in rewriting + * Record fail reason when in rewriting by struct info */ public void recordFailReason(StructInfo structInfo, String summary, Supplier failureReasonSupplier) { // record it's rewritten @@ -342,6 +343,24 @@ public void recordFailReason(StructInfo structInfo, String summary, Supplier failureReasonSupplier) { + // record it's rewritten + if (queryGroupPlan.getGroupExpression().isPresent()) { + this.addMatchedGroup(queryGroupPlan.getGroupExpression().get().getOwnerGroup().getGroupId(), + false); + } + // once success, do not record the fail reason + if (this.success) { + return; + } + this.failReason.put(queryGroupPlan.getGroupExpression() + .map(GroupExpression::getId).orElseGet(() -> new ObjectId(-1)), + Pair.of(summary, this.isEnableRecordFailureDetail() ? failureReasonSupplier.get() : "")); + } + @Override public String toString() { return getStringInfo(); @@ -410,6 +429,22 @@ public Void visitPhysicalRelation(PhysicalRelation physicalRelation, Void contex return builder.toString(); } + /** + * If materialized view rewrite duration is exceeded, make all materializationContexts with reason + * materialized view rewrite duration is exceeded + * */ + public static void makeFailWithDurationExceeded(Plan queryPlan, + List materializationContexts, long duration) { + for (MaterializationContext context : materializationContexts) { + if (context.isSuccess()) { + continue; + } + context.recordFailReason(queryPlan, + "materialized view rewrite duration is exceeded, the duration is " + duration, + () -> "materialized view rewrite duration is exceeded, the duration is " + duration); + } + } + private static String generateIdentifierName(List qualifiers) { return String.join(".", qualifiers); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java index 7107238a309219..42fc8701bf5baf 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java @@ -92,11 +92,15 @@ protected boolean checkMaterializationPattern(StructInfo structInfo, CascadesCon @Override protected Pair>, Map>> calcInvalidPartitions( - Plan queryPlan, Plan rewrittenPlan, AsyncMaterializationContext materializationContext, - CascadesContext cascadesContext) throws AnalysisException { + Set queryUsedPartition, + Plan rewrittenPlan, + CascadesContext cascadesContext, + AsyncMaterializationContext materializationContext) + throws AnalysisException { Pair>, Map>> invalidPartitions - = super.calcInvalidPartitions(queryPlan, rewrittenPlan, materializationContext, cascadesContext); - if (needUnionRewrite(invalidPartitions, cascadesContext)) { + = super.calcInvalidPartitions(queryUsedPartition, rewrittenPlan, cascadesContext, + materializationContext); + if (PartitionCompensator.needUnionRewrite(invalidPartitions, cascadesContext)) { // if query use some invalid partition in mv, bail out return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java new file mode 100644 index 00000000000000..98629f86028b81 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java @@ -0,0 +1,199 @@ +// 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.catalog.MTMV; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.PartitionInfo; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVPartitionInfo; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +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.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Handle materialized view partition union compensate handler + * */ +public class PartitionCompensator { + + public static final Logger LOG = LogManager.getLogger(PartitionCompensator.class); + // if partition pair is null which means can not get partitions from table in QueryPartitionCollector, + // we think this table scan query all partitions default + public static final Pair> ALL_PARTITIONS = Pair.of(null, null); + public static final Collection>> ALL_PARTITIONS_LIST = + ImmutableList.of(ALL_PARTITIONS); + + /** + * Maybe only some partitions is invalid in materialized view, or base table maybe add, modify, delete partition + * So we should calc the invalid partition used in query + * @param queryUsedBaseTablePartitionNameSet partitions used by query related partition table + * @param rewrittenPlan tmp rewrittenPlan when mv rewrite + * @param materializationContext the context of materialization,which hold materialized view meta and other info + * @param cascadesContext the context of cascades + * @return the key in pair is mvNeedRemovePartitionNameSet, the value in pair is baseTableNeedUnionPartitionNameSet + */ + public static Pair>, Map>> calcInvalidPartitions( + Set queryUsedBaseTablePartitionNameSet, Plan rewrittenPlan, + AsyncMaterializationContext materializationContext, CascadesContext cascadesContext) + throws AnalysisException { + Set mvNeedRemovePartitionNameSet = new HashSet<>(); + Set baseTableNeedUnionPartitionNameSet = new HashSet<>(); + // 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 Pair.of(ImmutableMap.of(), ImmutableMap.of()); + } + MTMVPartitionInfo mvCustomPartitionInfo = mtmv.getMvPartitionInfo(); + BaseTableInfo relatedPartitionTable = mvCustomPartitionInfo.getRelatedTableInfo(); + if (relatedPartitionTable == null || queryUsedBaseTablePartitionNameSet.isEmpty()) { + // if mv is not partitioned or query not query any partition, doesn't compensate + return Pair.of(ImmutableMap.of(), ImmutableMap.of()); + } + Collection mvValidPartitions = cascadesContext.getStatementContext() + .getMvCanRewritePartitionsMap().get(new BaseTableInfo(mtmv)); + Set mvValidPartitionNameSet = new HashSet<>(); + Set mvValidBaseTablePartitionNameSet = new HashSet<>(); + Set mvValidHasDataRelatedBaseTableNameSet = new HashSet<>(); + Pair>, Map> partitionMapping = mtmv.calculateDoublyPartitionMappings(); + for (Partition mvValidPartition : mvValidPartitions) { + mvValidPartitionNameSet.add(mvValidPartition.getName()); + Set relatedBaseTablePartitions = partitionMapping.key().get(mvValidPartition.getName()); + if (relatedBaseTablePartitions != null) { + mvValidBaseTablePartitionNameSet.addAll(relatedBaseTablePartitions); + } + if (!mtmv.selectNonEmptyPartitionIds(ImmutableList.of(mvValidPartition.getId())).isEmpty()) { + if (relatedBaseTablePartitions != null) { + mvValidHasDataRelatedBaseTableNameSet.addAll(relatedBaseTablePartitions); + } + } + } + if (Sets.intersection(mvValidHasDataRelatedBaseTableNameSet, queryUsedBaseTablePartitionNameSet).isEmpty()) { + // if mv can not offer any partition for query, query rewrite bail out + return null; + } + // Check when mv partition relates base table partition data change or delete partition + Set rewrittenPlanUsePartitionNameSet = new HashSet<>(); + List mvOlapScanList = rewrittenPlan.collectToList(node -> + node instanceof LogicalOlapScan + && Objects.equals(((CatalogRelation) node).getTable().getName(), mtmv.getName())); + for (Object olapScanObj : mvOlapScanList) { + LogicalOlapScan olapScan = (LogicalOlapScan) olapScanObj; + olapScan.getSelectedPartitionIds().forEach(id -> + rewrittenPlanUsePartitionNameSet.add(olapScan.getTable().getPartition(id).getName())); + } + // If rewritten plan use but not in mv valid partition name set, need remove in mv and base table union + Sets.difference(rewrittenPlanUsePartitionNameSet, mvValidPartitionNameSet) + .copyInto(mvNeedRemovePartitionNameSet); + for (String partitionName : mvNeedRemovePartitionNameSet) { + baseTableNeedUnionPartitionNameSet.addAll(partitionMapping.key().get(partitionName)); + } + // If related base table create partitions or mv is created with ttl, need base table union + Sets.difference(queryUsedBaseTablePartitionNameSet, mvValidBaseTablePartitionNameSet) + .copyInto(baseTableNeedUnionPartitionNameSet); + // Construct result map + Map> mvPartitionNeedRemoveNameMap = new HashMap<>(); + if (!mvNeedRemovePartitionNameSet.isEmpty()) { + mvPartitionNeedRemoveNameMap.put(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet); + } + Map> baseTablePartitionNeedUnionNameMap = new HashMap<>(); + if (!baseTableNeedUnionPartitionNameSet.isEmpty()) { + baseTablePartitionNeedUnionNameMap.put(relatedPartitionTable, baseTableNeedUnionPartitionNameSet); + } + return Pair.of(mvPartitionNeedRemoveNameMap, baseTablePartitionNeedUnionNameMap); + } + + public static boolean needUnionRewrite( + Pair>, Map>> invalidPartitions, + CascadesContext cascadesContext) { + return invalidPartitions != null + && (!invalidPartitions.key().isEmpty() || !invalidPartitions.value().isEmpty()); + } + + /** + * Check if need union compensate or not + */ + public static boolean needUnionRewrite(MaterializationContext materializationContext) { + if (!(materializationContext instanceof AsyncMaterializationContext)) { + return false; + } + MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); + PartitionType type = mtmv.getPartitionInfo().getType(); + BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); + return !PartitionType.UNPARTITIONED.equals(type) && relatedTableInfo != null; + } + + /** + * Get query used partitions + * this is calculated from tableUsedPartitionNameMap and tables in statementContext + * */ + public static Map, Set> getQueryUsedPartitions(StatementContext statementContext) { + // get table used partitions + // if table is not in statementContext().getTables() which means the table is partition prune as empty relation + Multimap, Pair>> tableUsedPartitionNameMap = statementContext + .getTableUsedPartitionNameMap(); + // if value is empty, means query no partitions + // if value is null, means query all partitions + // if value is not empty, means query some partitions + Map, Set> queryUsedRelatedTablePartitionsMap = new HashMap<>(); + outer: + for (Map.Entry, TableIf> queryUsedTableEntry : statementContext.getTables().entrySet()) { + Set usedPartitionSet = new HashSet<>(); + Collection>> tableUsedPartitions = + tableUsedPartitionNameMap.get(queryUsedTableEntry.getKey()); + if (!tableUsedPartitions.isEmpty()) { + if (ALL_PARTITIONS_LIST.equals(tableUsedPartitions)) { + queryUsedRelatedTablePartitionsMap.put(queryUsedTableEntry.getKey(), null); + continue; + } + for (Pair> partitionPair : tableUsedPartitions) { + if (ALL_PARTITIONS.equals(partitionPair)) { + queryUsedRelatedTablePartitionsMap.put(queryUsedTableEntry.getKey(), null); + continue outer; + } + usedPartitionSet.addAll(partitionPair.value()); + } + } + queryUsedRelatedTablePartitionsMap.put(queryUsedTableEntry.getKey(), usedPartitionSet); + } + return queryUsedRelatedTablePartitionsMap; + } +} 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 28983a04f0e028..2ffde75fe5b366 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 @@ -17,8 +17,6 @@ package org.apache.doris.nereids.rules.exploration.mv; -import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Pair; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; @@ -33,7 +31,6 @@ import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; import org.apache.doris.nereids.trees.expressions.EqualTo; -import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; @@ -50,7 +47,6 @@ import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand.PredicateAddContext; import org.apache.doris.nereids.trees.plans.commands.UpdateMvByPartitionCommand.PredicateAdder; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; -import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; 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.LogicalOlapScan; @@ -60,12 +56,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; -import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; @@ -131,8 +124,6 @@ public class StructInfo { // this is for building LogicalCompatibilityContext later. private final Map> expressionToShuttledExpressionToMap; - // Record the exprId and the corresponding expr map, this is used by expression shuttled - private final Map namedExprIdAndExprMapping; private final List planOutputShuttledExpressions; /** @@ -145,7 +136,6 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG Map>> shuttledExpressionsToExpressionsMap, Map> expressionToShuttledExpressionToMap, - Map namedExprIdAndExprMapping, BitSet tableIdSet, SplitPredicate splitPredicate, EquivalenceClass equivalenceClass, @@ -164,7 +154,6 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG this.equivalenceClass = equivalenceClass; this.shuttledExpressionsToExpressionsMap = shuttledExpressionsToExpressionsMap; this.expressionToShuttledExpressionToMap = expressionToShuttledExpressionToMap; - this.namedExprIdAndExprMapping = namedExprIdAndExprMapping; this.planOutputShuttledExpressions = planOutputShuttledExpressions; } @@ -175,8 +164,7 @@ public StructInfo withPredicates(Predicates predicates) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, predicates, this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, - this.namedExprIdAndExprMapping, this.tableBitSet, - null, null, this.planOutputShuttledExpressions); + this.tableBitSet, null, null, this.planOutputShuttledExpressions); } /** @@ -186,8 +174,7 @@ public StructInfo withTableBitSet(BitSet tableBitSet) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, this.predicates, this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, - this.namedExprIdAndExprMapping, tableBitSet, - this.splitPredicate, this.equivalenceClass, this.planOutputShuttledExpressions); + tableBitSet, this.splitPredicate, this.equivalenceClass, this.planOutputShuttledExpressions); } private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, @@ -195,7 +182,6 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, Map>> shuttledExpressionsToExpressionsMap, Map> expressionToShuttledExpressionToMap, - Map namedExprIdAndExprMapping, List relations, Map relationIdStructInfoNodeMap, BitSet hyperTableBitSet, @@ -213,22 +199,16 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, // plan relation collector and set to map StructInfoNode structInfoNode = (StructInfoNode) node; // record expressions in node - if (structInfoNode.getExpressions() != null) { - structInfoNode.getExpressions().forEach(expression -> { - ExpressionLineageReplacer.ExpressionReplaceContext replaceContext = - new ExpressionLineageReplacer.ExpressionReplaceContext( - Lists.newArrayList(expression), ImmutableSet.of(), - ImmutableSet.of(), new BitSet()); - structInfoNode.getPlan().accept(ExpressionLineageReplacer.INSTANCE, replaceContext); - // Replace expressions by expression map - List replacedExpressions = replaceContext.getReplacedExpressions(); + List nodeExpressions = structInfoNode.getExpressions(); + if (nodeExpressions != null) { + List shuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( + nodeExpressions, structInfoNode.getPlan(), + new BitSet()); + for (int index = 0; index < nodeExpressions.size(); index++) { putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, expressionToShuttledExpressionToMap, - ExpressionPosition.NODE, replacedExpressions.get(0), expression, node); - // Record this, will be used in top level expression shuttle later, see the method - // ExpressionLineageReplacer#visitGroupPlan - namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); - }); + ExpressionPosition.NODE, shuttledExpressions.get(index), nodeExpressions.get(index), node); + } } // every node should only have one relation, this is for LogicalCompatibilityContext if (!nodeRelations.isEmpty()) { @@ -240,37 +220,27 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, List joinConjunctExpressions = edge.getExpressions(); // shuttle expression in edge for the build of LogicalCompatibilityContext later. // Record the exprId to expr map in the processing to strut info - // TODO get exprId to expr map when complex project is ready in join dege - ExpressionLineageReplacer.ExpressionReplaceContext replaceContext = - new ExpressionLineageReplacer.ExpressionReplaceContext( - joinConjunctExpressions.stream().map(expr -> (Expression) expr) - .collect(Collectors.toList()), - ImmutableSet.of(), ImmutableSet.of(), new BitSet()); - topPlan.accept(ExpressionLineageReplacer.INSTANCE, replaceContext); // Replace expressions by expression map - List replacedExpressions = replaceContext.getReplacedExpressions(); - for (int i = 0; i < replacedExpressions.size(); i++) { + List shuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( + joinConjunctExpressions, topPlan, new BitSet()); + for (int i = 0; i < shuttledExpressions.size(); i++) { putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, expressionToShuttledExpressionToMap, - ExpressionPosition.JOIN_EDGE, replacedExpressions.get(i), joinConjunctExpressions.get(i), - edge); + ExpressionPosition.JOIN_EDGE, shuttledExpressions.get(i), + joinConjunctExpressions.get(i), edge); } - // Record this, will be used in top level expression shuttle later, see the method - // ExpressionLineageReplacer#visitGroupPlan - namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); } // Collect expression from where in hyper graph hyperGraph.getFilterEdges().forEach(filterEdge -> { List filterExpressions = filterEdge.getExpressions(); - filterExpressions.forEach(predicate -> { - // this is used for LogicalCompatibilityContext - ExpressionUtils.extractConjunction(predicate).forEach(expr -> - putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, - expressionToShuttledExpressionToMap, - ExpressionPosition.FILTER_EDGE, - ExpressionUtils.shuttleExpressionWithLineage(predicate, topPlan, new BitSet()), - predicate, filterEdge)); - }); + List shuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( + filterExpressions, topPlan, new BitSet()); + for (int i = 0; i < shuttledExpressions.size(); i++) { + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, + ExpressionPosition.FILTER_EDGE, shuttledExpressions.get(i), + filterExpressions.get(i), filterEdge); + } }); return true; } @@ -343,12 +313,10 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable Map relationIdStructInfoNodeMap = new LinkedHashMap<>(); Map>> shuttledHashConjunctsToConjunctsMap = new LinkedHashMap<>(); - Map namedExprIdAndExprMapping = new LinkedHashMap<>(); BitSet tableBitSet = new BitSet(); Map> expressionToShuttledExpressionToMap = new HashMap<>(); boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap, expressionToShuttledExpressionToMap, - namedExprIdAndExprMapping, relationList, relationIdStructInfoNodeMap, tableBitSet, @@ -370,7 +338,7 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable return new StructInfo(originalPlan, originalPlanId, hyperGraph, valid, topPlan, bottomPlan, relationList, relationIdStructInfoNodeMap, predicates, shuttledHashConjunctsToConjunctsMap, expressionToShuttledExpressionToMap, - namedExprIdAndExprMapping, tableBitSet, null, null, + tableBitSet, null, null, planOutputShuttledExpressions); } @@ -471,10 +439,6 @@ public ObjectId getOriginalPlanId() { return originalPlanId; } - public Map getNamedExprIdAndExprMapping() { - return namedExprIdAndExprMapping; - } - public BitSet getTableBitSet() { return tableBitSet; } @@ -767,35 +731,6 @@ public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, } } - /** - * Collect partitions on base table - */ - public static class QueryScanPartitionsCollector extends DefaultPlanVisitor>> { - @Override - public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, - Map> targetTablePartitionMap) { - TableIf table = catalogRelation.getTable(); - BaseTableInfo relatedPartitionTable = new BaseTableInfo(table); - if (!targetTablePartitionMap.containsKey(relatedPartitionTable)) { - return catalogRelation; - } - if (catalogRelation instanceof LogicalOlapScan) { - // Handle olap table - LogicalOlapScan logicalOlapScan = (LogicalOlapScan) catalogRelation; - Set tablePartitions = targetTablePartitionMap.get(relatedPartitionTable); - for (Long partitionId : logicalOlapScan.getSelectedPartitionIds()) { - tablePartitions.add(logicalOlapScan.getTable().getPartition(partitionId)); - } - } else { - // todo Support other type partition table - // Not support to partition check now when query external catalog table, support later. - targetTablePartitionMap.clear(); - } - return catalogRelation; - } - } - /** * Add filter on table scan according to table filter map * diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java index e99906f5e13dc4..89dfb75f2ebbad 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PruneFileScanPartition.java @@ -64,7 +64,6 @@ public Rule build() { // set isPruned so that it won't go pass the partition prune again selectedPartitions = new SelectedPartitions(0, ImmutableMap.of(), true); } - LogicalFileScan rewrittenScan = scan.withSelectedPartitions(selectedPartitions); return new LogicalFilter<>(filter.getConjuncts(), rewrittenScan); }).toRule(RuleType.FILE_SCAN_PARTITION_PRUNE); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/QueryPartitionCollector.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/QueryPartitionCollector.java new file mode 100644 index 00000000000000..2ad993b361d43a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/QueryPartitionCollector.java @@ -0,0 +1,95 @@ +// 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.rewrite; + +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.ExternalTable; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.rules.exploration.mv.PartitionCompensator; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan.SelectedPartitions; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.Multimap; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Used to collect query partitions, only collect once + * */ +public class QueryPartitionCollector extends DefaultPlanRewriter implements CustomRewriter { + + public static final Logger LOG = LogManager.getLogger(QueryPartitionCollector.class); + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + + ConnectContext connectContext = ConnectContext.get(); + if (connectContext != null && connectContext.getSessionVariable().internalSession) { + return plan; + } + plan.accept(this, connectContext); + return plan; + } + + @Override + public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, ConnectContext context) { + + TableIf table = catalogRelation.getTable(); + if (table.getDatabase() == null) { + LOG.error("QueryPartitionCollector visitLogicalCatalogRelation database is null, table is " + + table.getName()); + return catalogRelation; + } + Multimap, Pair>> tableUsedPartitionNameMap = context.getStatementContext() + .getTableUsedPartitionNameMap(); + Set tablePartitions = new HashSet<>(); + if (catalogRelation instanceof LogicalOlapScan) { + // Handle olap table + LogicalOlapScan logicalOlapScan = (LogicalOlapScan) catalogRelation; + for (Long partitionId : logicalOlapScan.getSelectedPartitionIds()) { + tablePartitions.add(logicalOlapScan.getTable().getPartition(partitionId).getName()); + } + tableUsedPartitionNameMap.put(table.getFullQualifiers(), + Pair.of(catalogRelation.getRelationId(), tablePartitions)); + } else if (catalogRelation instanceof LogicalFileScan + && catalogRelation.getTable() != null + && ((ExternalTable) catalogRelation.getTable()).supportInternalPartitionPruned()) { + LogicalFileScan logicalFileScan = (LogicalFileScan) catalogRelation; + SelectedPartitions selectedPartitions = logicalFileScan.getSelectedPartitions(); + tablePartitions.addAll(selectedPartitions.selectedPartitions.keySet()); + tableUsedPartitionNameMap.put(table.getFullQualifiers(), + Pair.of(catalogRelation.getRelationId(), tablePartitions)); + } else { + // not support get partition scene, we consider query all partitions from table + tableUsedPartitionNameMap.put(table.getFullQualifiers(), PartitionCompensator.ALL_PARTITIONS); + } + return catalogRelation; + } +} 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 1252f3b4bbf899..b9f1cc907c538b 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 @@ -18,8 +18,6 @@ package org.apache.doris.nereids.trees.plans.visitor; import org.apache.doris.catalog.TableIf.TableType; -import org.apache.doris.nereids.memo.Group; -import org.apache.doris.nereids.rules.exploration.mv.StructInfo; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; @@ -31,12 +29,14 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer.ExpressionReplaceContext; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + import java.util.ArrayList; import java.util.BitSet; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -47,6 +47,7 @@ */ public class ExpressionLineageReplacer extends DefaultPlanVisitor { + public static final Logger LOG = LogManager.getLogger(ExpressionLineageReplacer.class); public static final ExpressionLineageReplacer INSTANCE = new ExpressionLineageReplacer(); @Override @@ -63,25 +64,7 @@ public Expression visit(Plan plan, ExpressionReplaceContext context) { @Override public Expression visitGroupPlan(GroupPlan groupPlan, ExpressionReplaceContext context) { - Group group = groupPlan.getGroup(); - if (group == null) { - return visit(groupPlan, context); - } - Collection structInfos = group.getstructInfoMap().getStructInfos(); - if (structInfos.isEmpty()) { - return visit(groupPlan, context); - } - // Find first info which the context's bitmap contains all to make sure that - // the expression lineage is correct - Optional structInfoOptional = structInfos.stream() - .filter(info -> (context.getTableBitSet().isEmpty() - || StructInfo.containsAll(context.getTableBitSet(), info.getTableBitSet())) - && !info.getNamedExprIdAndExprMapping().isEmpty()) - .findFirst(); - if (!structInfoOptional.isPresent()) { - return visit(groupPlan, context); - } - context.getExprIdExpressionMap().putAll(structInfoOptional.get().getNamedExprIdAndExprMapping()); + LOG.error("ExpressionLineageReplacer should not meet groupPlan, plan is {}", groupPlan.toString()); return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index e73ab63f7a8770..64b0590cc739c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -589,6 +589,9 @@ public class SessionVariable implements Serializable, Writable { public static final String CREATE_TABLE_PARTITION_MAX_NUM = "create_table_partition_max_num"; + public static final String MATERIALIZED_VIEW_REWRITE_DURATION_THRESHOLD_MS + = "materialized_view_rewrite_duration_threshold_ms"; + public static final String ENABLE_PUSHDOWN_MINMAX_ON_UNIQUE = "enable_pushdown_minmax_on_unique"; public static final String HIVE_PARQUET_USE_COLUMN_NAMES = "hive_parquet_use_column_names"; @@ -1979,6 +1982,12 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { "Whether enable materialized view nest rewrite"}) public boolean enableMaterializedViewNestRewrite = false; + @VariableMgr.VarAttr(name = MATERIALIZED_VIEW_REWRITE_DURATION_THRESHOLD_MS, needForward = true, + description = {"物化视图透明改写允许的最长耗时,超过此时长不再进行透明改写", + "The maximum duration allowed for transparent rewriting of materialized views; " + + "if this duration is exceeded, transparent rewriting will no longer be performed."}) + public long materializedViewRewriteDurationThresholdMs = 1000L; + @VariableMgr.VarAttr(name = CREATE_TABLE_PARTITION_MAX_NUM, needForward = true, description = {"建表时创建分区的最大数量", "The maximum number of partitions created during table creation"}) diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java index 25ad6b05dd8454..82c7eaac631e81 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java @@ -27,6 +27,7 @@ import org.apache.doris.qe.SessionVariable; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Mocked; import org.junit.Assert; @@ -64,6 +65,14 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = Lists.newArrayList(p1); + mtmv.getPartitionNames(); + minTimes = 0; + result = Sets.newHashSet("p1"); + + p1.getName(); + minTimes = 0; + result = "p1"; + p1.getVisibleVersionTime(); minTimes = 0; result = 1L; @@ -136,14 +145,15 @@ public void testGetMTMVCanRewritePartitionsForceConsistent() throws AnalysisExce // currentTimeMills is 3, grace period is 2, and partition getVisibleVersionTime is 1 // if forceConsistent this should get 0 partitions which mtmv can use. Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, true); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, true, null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } @Test public void testGetMTMVCanRewritePartitionsNormal() { Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(1, mtmvCanRewritePartitions.size()); } @@ -164,7 +174,8 @@ public void testGetMTMVCanRewritePartitionsInGracePeriod() throws AnalysisExcept }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(1, mtmvCanRewritePartitions.size()); } @@ -185,7 +196,8 @@ public void testGetMTMVCanRewritePartitionsNotInGracePeriod() throws AnalysisExc }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } @@ -199,7 +211,8 @@ public void testGetMTMVCanRewritePartitionsDisableMaterializedViewRewrite() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); // getMTMVCanRewritePartitions only check the partition is valid or not, it doesn't care the // isEnableMaterializedViewRewriteWhenBaseTableUnawareness Assert.assertEquals(1, mtmvCanRewritePartitions.size()); @@ -217,7 +230,8 @@ public void testGetMTMVCanRewritePartitionsNotSync() throws AnalysisException { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } @@ -235,7 +249,8 @@ public void testGetMTMVCanRewritePartitionsEnableContainExternalTable() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(1, mtmvCanRewritePartitions.size()); } @@ -253,7 +268,8 @@ public void testGetMTMVCanRewritePartitionsDisableContainExternalTable() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); // getMTMVCanRewritePartitions only check the partition is valid or not, it doesn't care the // isEnableMaterializedViewRewriteWhenBaseTableUnawareness Assert.assertEquals(1, mtmvCanRewritePartitions.size()); @@ -269,7 +285,8 @@ public void testGetMTMVCanRewritePartitionsStateAbnormal() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } @@ -283,7 +300,8 @@ public void testGetMTMVCanRewritePartitionsRefreshStateAbnormal() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(1, mtmvCanRewritePartitions.size()); } @@ -297,7 +315,8 @@ public void testGetMTMVCanRewritePartitionsRefreshStateInit() { } }; Collection mtmvCanRewritePartitions = MTMVRewriteUtil - .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false); + .getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false, + null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java index 6b10176e22aa9c..19d1efdbbd82d6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java @@ -65,7 +65,8 @@ public BitSet getDisableNereidsRules() { Assertions.assertEquals(1, tableMaps.size()); new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, + Set relatedPartitions) { return true; } }; @@ -123,7 +124,8 @@ public BitSet getDisableNereidsRules() { Assertions.assertEquals(1, tableMaps.size()); new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, + Set relatedPartitions) { return true; } }; @@ -147,7 +149,6 @@ public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPar .printlnBestPlanTree(); root = c1.getMemo().getRoot(); root.getstructInfoMap().refresh(root, c1, new HashSet<>()); - root.getstructInfoMap().refresh(root, c1, new HashSet<>()); tableMaps = root.getstructInfoMap().getTableMaps(); Assertions.assertEquals(2, tableMaps.size()); dropMvByNereids("drop materialized view mv1"); @@ -171,7 +172,8 @@ public BitSet getDisableNereidsRules() { ); new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, + Set relatedPartitions) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java index 0090982db00898..1403a9fee5e254 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java @@ -54,7 +54,8 @@ public BitSet getDisableNereidsRules() { }; new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, + Set queryUsedRelatedTablePartitionsMap) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MtmvCacheNewConnectContextTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MtmvCacheNewConnectContextTest.java index 0134d5df4e7166..a15aad19de4c77 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MtmvCacheNewConnectContextTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MtmvCacheNewConnectContextTest.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.mv; import org.apache.doris.catalog.MTMV; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRelationManager; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.sqltest.SqlTestBase; @@ -31,6 +32,8 @@ import org.junit.jupiter.api.Test; import java.util.BitSet; +import java.util.Map; +import java.util.Set; /** * The connectContext would new instance when generate MTMVCache, after generate, the connectContext should @@ -52,7 +55,8 @@ public BitSet getDisableNereidsRules() { }; new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, + Map> queryUsedRelatedTablePartitionsMap) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java index dd15b5e06c7899..4fa0a68e77c6ab 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java @@ -31,6 +31,7 @@ import org.junit.jupiter.api.Test; import java.util.BitSet; +import java.util.Set; /** * Test mv rewrite when base table id is lager then integer @@ -49,7 +50,8 @@ public BitSet getDisableNereidsRules() { }; new MockUp() { @Mock - public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) { + public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, + Set queryUsedRelatedTablePartitionsMap) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/OptimizeGetAvailableMvsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/OptimizeGetAvailableMvsTest.java new file mode 100644 index 00000000000000..da327b2ba6ca60 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/OptimizeGetAvailableMvsTest.java @@ -0,0 +1,269 @@ +// 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.mv; + +import org.apache.doris.catalog.DistributionInfo; +import org.apache.doris.catalog.MaterializedIndex; +import org.apache.doris.catalog.MaterializedIndex.IndexState; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.rules.expression.rules.PartitionPruner; +import org.apache.doris.nereids.rules.expression.rules.PartitionPruner.PartitionTableType; +import org.apache.doris.nereids.sqltest.SqlTestBase; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.qe.SessionVariable; + +import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; +import mockit.Mock; +import mockit.MockUp; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.BitSet; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Test get available mvs after rewrite by rules + */ +public class OptimizeGetAvailableMvsTest extends SqlTestBase { + + @Test + void testWhenNotPartitionPrune() throws Exception { + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + BitSet disableNereidsRules = connectContext.getSessionVariable().getDisableNereidsRules(); + new MockUp() { + @Mock + public BitSet getDisableNereidsRules() { + return disableNereidsRules; + } + }; + + new MockUp() { + @Mock + public Partition getPartition(long partitionId) { + return new Partition() { + @Override + public long getId() { + return 1L; + } + + @Override + public String getName() { + return "mock_partition"; + } + + @Override + public PartitionState getState() { + return PartitionState.NORMAL; + } + + @Override + public MaterializedIndex getIndex(long indexId) { + return new MaterializedIndex(1L, IndexState.NORMAL); + } + + @Override + public DistributionInfo getDistributionInfo() { + return new DistributionInfo() { + @Override + public DistributionInfoType getType() { + return DistributionInfoType.RANDOM; + } + }; + } + }; + } + }; + + new MockUp() { + @Mock + public List getSelectedPartitionIds() { + return Lists.newArrayList(1L); + } + }; + + connectContext.getSessionVariable().enableMaterializedViewRewrite = true; + connectContext.getSessionVariable().enableMaterializedViewNestRewrite = true; + createMvByNereids("create materialized view mv1 " + + " BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL\n" + + " PARTITION BY (id)\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as " + + " select T4.id from T4 inner join T2 " + + " on T4.id = T2.id;"); + CascadesContext c1 = createCascadesContext( + "select T4.id " + + "from T4 " + + "inner join T2 on T4.id = T2.id " + + "inner join T3 on T4.id = T3.id", + connectContext + ); + PlanChecker.from(c1) + .analyze() + .rewrite() + .optimize() + .printlnBestPlanTree(); + Multimap, Pair>> tableUsedPartitionNameMap = c1.getStatementContext() + .getTableUsedPartitionNameMap(); + Map> mvCanRewritePartitionsMap = c1.getStatementContext() + .getMvCanRewritePartitionsMap(); + Assertions.assertFalse(tableUsedPartitionNameMap.isEmpty()); + + for (Map.Entry, Pair>> tableInfoEntry + : tableUsedPartitionNameMap.entries()) { + if (tableInfoEntry.getKey().contains("T2")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } else if (tableInfoEntry.getKey().contains("T3")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } else if (tableInfoEntry.getKey().contains("T4")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } + } + + Assertions.assertEquals(1, mvCanRewritePartitionsMap.size()); + Assertions.assertTrue(mvCanRewritePartitionsMap.keySet().iterator().next().getTableName() + .equalsIgnoreCase("mv1")); + + dropMvByNereids("drop materialized view mv1"); + } + + @Test + void testWhenPartitionPrune() throws Exception { + connectContext.getSessionVariable().setDisableNereidsRules("PRUNE_EMPTY_PARTITION"); + BitSet disableNereidsRules = connectContext.getSessionVariable().getDisableNereidsRules(); + new MockUp() { + @Mock + public BitSet getDisableNereidsRules() { + return disableNereidsRules; + } + }; + + new MockUp() { + @Mock + public > List prune(List partitionSlots, Expression partitionPredicate, + Map idToPartitions, CascadesContext cascadesContext, + PartitionTableType partitionTableType) { + return Lists.newArrayList(1L); + } + }; + + new MockUp() { + @Mock + public Partition getPartition(long partitionId) { + return new Partition() { + @Override + public long getId() { + return 1L; + } + + @Override + public String getName() { + return "mock_partition"; + } + + @Override + public PartitionState getState() { + return PartitionState.NORMAL; + } + + @Override + public MaterializedIndex getIndex(long indexId) { + return new MaterializedIndex(1L, IndexState.NORMAL); + } + + @Override + public DistributionInfo getDistributionInfo() { + return new DistributionInfo() { + @Override + public DistributionInfoType getType() { + return DistributionInfoType.RANDOM; + } + }; + } + }; + } + }; + + new MockUp() { + @Mock + public List getSelectedPartitionIds() { + return Lists.newArrayList(1L); + } + }; + + connectContext.getSessionVariable().enableMaterializedViewRewrite = true; + connectContext.getSessionVariable().enableMaterializedViewNestRewrite = true; + createMvByNereids("create materialized view mv2 " + + " BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL\n" + + " PARTITION BY (id)\n" + + " DISTRIBUTED BY RANDOM BUCKETS 1\n" + + " PROPERTIES ('replication_num' = '1') \n" + + " as " + + " select T4.id from T4 inner join T2 " + + " on T4.id = T2.id;"); + CascadesContext c1 = createCascadesContext( + "select T4.id " + + "from T4 " + + "inner join T2 on T4.id = T2.id " + + "inner join T3 on T4.id = T3.id " + + "where T4.id > 0", + connectContext + ); + PlanChecker.from(c1) + .analyze() + .rewrite() + .optimize() + .printlnBestPlanTree(); + Multimap, Pair>> tableUsedPartitionNameMap = c1.getStatementContext() + .getTableUsedPartitionNameMap(); + Map> mvCanRewritePartitionsMap = c1.getStatementContext() + .getMvCanRewritePartitionsMap(); + Assertions.assertFalse(tableUsedPartitionNameMap.isEmpty()); + + for (Map.Entry, Pair>> tableInfoEntry + : tableUsedPartitionNameMap.entries()) { + if (tableInfoEntry.getKey().contains("T2")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } else if (tableInfoEntry.getKey().contains("T3")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } else if (tableInfoEntry.getKey().contains("T4")) { + Assertions.assertEquals(tableInfoEntry.getValue().value(), Sets.newHashSet("mock_partition")); + } + } + + Assertions.assertEquals(1, mvCanRewritePartitionsMap.size()); + Assertions.assertTrue(mvCanRewritePartitionsMap.keySet().iterator().next().getTableName() + .equalsIgnoreCase("mv2")); + + dropMvByNereids("drop materialized view mv2"); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensatorTest.java new file mode 100644 index 00000000000000..6588d4abd86665 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensatorTest.java @@ -0,0 +1,186 @@ +// 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.common.Pair; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.util.PlanChecker; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multimap; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class PartitionCompensatorTest extends TestWithFeService { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("partition_compensate_test"); + useDatabase("partition_compensate_test"); + + createTable("CREATE TABLE `lineitem_list_partition` (\n" + + " `l_orderkey` BIGINT not NULL,\n" + + " `l_linenumber` INT NULL,\n" + + " `l_partkey` INT NULL,\n" + + " `l_suppkey` INT NULL,\n" + + " `l_quantity` DECIMAL(15, 2) NULL,\n" + + " `l_extendedprice` DECIMAL(15, 2) NULL,\n" + + " `l_discount` DECIMAL(15, 2) NULL,\n" + + " `l_tax` DECIMAL(15, 2) NULL,\n" + + " `l_returnflag` VARCHAR(1) NULL,\n" + + " `l_linestatus` VARCHAR(1) NULL,\n" + + " `l_commitdate` DATE NULL,\n" + + " `l_receiptdate` DATE NULL,\n" + + " `l_shipinstruct` VARCHAR(25) NULL,\n" + + " `l_shipmode` VARCHAR(10) NULL,\n" + + " `l_comment` VARCHAR(44) NULL,\n" + + " `l_shipdate` DATE NULL\n" + + " ) ENGINE=OLAP\n" + + " DUPLICATE KEY(l_orderkey, l_linenumber, l_partkey, l_suppkey )\n" + + " COMMENT 'OLAP'\n" + + " PARTITION BY list(l_orderkey) (\n" + + " PARTITION p1 VALUES in ('1'),\n" + + " PARTITION p2 VALUES in ('2'),\n" + + " PARTITION p3 VALUES in ('3')\n" + + " )\n" + + " DISTRIBUTED BY HASH(`l_orderkey`) BUCKETS 3\n" + + " PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + " )"); + + createTable("CREATE TABLE `orders_list_partition` (\n" + + " `o_orderkey` BIGINT not NULL,\n" + + " `o_custkey` INT NULL,\n" + + " `o_orderstatus` VARCHAR(1) NULL,\n" + + " `o_totalprice` DECIMAL(15, 2) NULL,\n" + + " `o_orderpriority` VARCHAR(15) NULL,\n" + + " `o_clerk` VARCHAR(15) NULL,\n" + + " `o_shippriority` INT NULL,\n" + + " `o_comment` VARCHAR(79) NULL,\n" + + " `o_orderdate` DATE NULL\n" + + " ) ENGINE=OLAP\n" + + " DUPLICATE KEY(`o_orderkey`, `o_custkey`)\n" + + " COMMENT 'OLAP'\n" + + " PARTITION BY list(o_orderkey) (\n" + + " PARTITION p1 VALUES in ('1'),\n" + + " PARTITION p2 VALUES in ('2'),\n" + + " PARTITION p3 VALUES in ('3'),\n" + + " PARTITION p4 VALUES in ('4')\n" + + " )\n" + + " DISTRIBUTED BY HASH(`o_orderkey`) BUCKETS 3\n" + + " PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + " )"); + + // Should not make scan to empty relation when the table used by materialized view has no data + connectContext.getSessionVariable().setDisableNereidsRules( + "OLAP_SCAN_PARTITION_PRUNE,PRUNE_EMPTY_PARTITION"); + } + + // Test when join both side are all partition table and partition column name is same + @Test + public void testGetQueryTableUsedPartition() { + PlanChecker.from(connectContext) + .checkExplain("select l1.*, O_CUSTKEY \n" + + "from lineitem_list_partition l1\n" + + "left outer join orders_list_partition\n" + + "on l1.l_shipdate = o_orderdate\n", + nereidsPlanner -> { + Map, Set> queryUsedPartitions + = PartitionCompensator.getQueryUsedPartitions( + nereidsPlanner.getCascadesContext().getStatementContext()); + + List itmeQualifier = ImmutableList.of( + "internal", "partition_compensate_test", "lineitem_list_partition"); + Set queryTableUsedPartition = queryUsedPartitions.get(itmeQualifier); + Assertions.assertEquals(queryTableUsedPartition, ImmutableSet.of("p1", "p2", "p3")); + + List orderQualifier = ImmutableList.of( + "internal", "partition_compensate_test", "orders_list_partition"); + Set orderTableUsedPartition = queryUsedPartitions.get(orderQualifier); + Assertions.assertEquals(orderTableUsedPartition, ImmutableSet.of("p1", "p2", "p3", "p4")); + }); + } + + @Test + public void testGetAllTableUsedPartition() { + PlanChecker.from(connectContext) + .checkExplain("select l1.*, O_CUSTKEY \n" + + "from lineitem_list_partition l1\n" + + "left outer join orders_list_partition\n" + + "on l1.l_shipdate = o_orderdate\n", + nereidsPlanner -> { + List qualifier = ImmutableList.of( + "internal", "partition_compensate_test", "lineitem_list_partition"); + + Multimap, Pair>> tableUsedPartitionNameMap + = connectContext.getStatementContext().getTableUsedPartitionNameMap(); + tableUsedPartitionNameMap.put(qualifier, PartitionCompensator.ALL_PARTITIONS); + + Map, Set> queryUsedPartitions + = PartitionCompensator.getQueryUsedPartitions( + nereidsPlanner.getCascadesContext().getStatementContext()); + Set queryTableUsedPartition = queryUsedPartitions.get(qualifier); + // if tableUsedPartitionNameMap contain any PartitionCompensator.ALL_PARTITIONS + // consider query all partitions from table + Assertions.assertNull(queryTableUsedPartition); + + List orderQualifier = ImmutableList.of( + "internal", "partition_compensate_test", "orders_list_partition"); + Set orderTableUsedPartition = queryUsedPartitions.get(orderQualifier); + Assertions.assertEquals(orderTableUsedPartition, ImmutableSet.of("p1", "p2", "p3", "p4")); + }); + } + + @Test + public void testGetAllTableUsedPartitionList() { + PlanChecker.from(connectContext) + .checkExplain("select l1.*, O_CUSTKEY \n" + + "from lineitem_list_partition l1\n" + + "left outer join orders_list_partition\n" + + "on l1.l_shipdate = o_orderdate\n", + nereidsPlanner -> { + List qualifier = ImmutableList.of( + "internal", "partition_compensate_test", "lineitem_list_partition"); + + Multimap, Pair>> tableUsedPartitionNameMap + = connectContext.getStatementContext().getTableUsedPartitionNameMap(); + tableUsedPartitionNameMap.removeAll(qualifier); + tableUsedPartitionNameMap.put(qualifier, PartitionCompensator.ALL_PARTITIONS); + + Map, Set> queryUsedPartitions + = PartitionCompensator.getQueryUsedPartitions( + nereidsPlanner.getCascadesContext().getStatementContext()); + Set queryTableUsedPartition = queryUsedPartitions.get(qualifier); + // if tableUsedPartitionNameMap contain only PartitionCompensator.ALL_PARTITIONS + // consider query all partitions from table + Assertions.assertNull(queryTableUsedPartition); + + List orderQualifier = ImmutableList.of( + "internal", "partition_compensate_test", "orders_list_partition"); + Set orderTableUsedPartition = queryUsedPartitions.get(orderQualifier); + Assertions.assertEquals(orderTableUsedPartition, ImmutableSet.of("p1", "p2", "p3", "p4")); + }); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java index e0cf7f66a02fe7..e900937d2baa68 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java @@ -121,7 +121,6 @@ public PlanChecker parse(String sql) { public PlanChecker analyze() { this.cascadesContext.newAnalyzer().analyze(); this.cascadesContext.toMemo(); - InitMaterializationContextHook.INSTANCE.initMaterializationContext(this.cascadesContext); return this; } @@ -245,6 +244,8 @@ public Rule build() { public PlanChecker rewrite() { Rewriter.getWholeTreeRewriter(cascadesContext).execute(); + cascadesContext.newTablePartitionCollector().execute(); + InitMaterializationContextHook.INSTANCE.initMaterializationContext(this.cascadesContext); cascadesContext.toMemo(); return this; } @@ -553,8 +554,10 @@ public void onInvokeRule(RuleType ruleType) { public PlanChecker checkExplain(String sql, Consumer consumer) { LogicalPlan parsed = new NereidsParser().parseSingle(sql); + StatementContext statementContext = new StatementContext(connectContext, new OriginStatement(sql, 0)); NereidsPlanner nereidsPlanner = new NereidsPlanner( - new StatementContext(connectContext, new OriginStatement(sql, 0))); + statementContext); + connectContext.setStatementContext(statementContext); LogicalPlanAdapter adapter = LogicalPlanAdapter.of(parsed); adapter.setIsExplain(new ExplainOptions(ExplainLevel.ALL_PLAN, false)); nereidsPlanner.plan(adapter); @@ -564,8 +567,9 @@ public PlanChecker checkExplain(String sql, Consumer consumer) { public PlanChecker checkPlannerResult(String sql, Consumer consumer) { LogicalPlan parsed = new NereidsParser().parseSingle(sql); - NereidsPlanner nereidsPlanner = new NereidsPlanner( - new StatementContext(connectContext, new OriginStatement(sql, 0))); + StatementContext statementContext = new StatementContext(connectContext, new OriginStatement(sql, 0)); + NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); + connectContext.setStatementContext(statementContext); nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); consumer.accept(nereidsPlanner); return this; diff --git a/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.out b/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.out new file mode 100644 index 00000000000000..9c3b0f5506395d --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.out @@ -0,0 +1,143 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query_3_0_before -- +2023-10-17 2023-10-17 2 3 1990.00 +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_3_0_after -- +2023-10-17 2023-10-17 2 3 1990.00 +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_4_0_before -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_4_0_after -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_7_0_before -- +2023-10-17 2023-10-17 2 3 1990.00 +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_7_0_after -- +2023-10-17 2023-10-17 2 3 1990.00 +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_8_0_before -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_8_0_after -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_11_0_before -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_11_0_after -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_12_0_before -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_12_0_after -- +2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_16_0_before -- +2023-10-18 2023-10-18 2 3 436.80 +2023-10-19 2023-10-19 2 3 398.00 + +-- !query_16_0_after -- +2023-10-18 2023-10-18 2 3 436.80 +2023-10-19 2023-10-19 2 3 398.00 + +-- !query_17_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_17_0_after -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_18_0_before -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_18_0_after -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_19_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-21 2023-11-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_19_0_after -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-21 2023-11-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_20_0_before -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_20_0_after -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_21_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_21_0_after -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_22_0_before -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_22_0_after -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + diff --git a/regression-test/data/nereids_rules_p0/mv/partition_mv_rewrite.out b/regression-test/data/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.out similarity index 69% rename from regression-test/data/nereids_rules_p0/mv/partition_mv_rewrite.out rename to regression-test/data/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.out index c6b66a59749cae..1e718b4e6127d3 100644 --- a/regression-test/data/nereids_rules_p0/mv/partition_mv_rewrite.out +++ b/regression-test/data/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.out @@ -3,11 +3,13 @@ 2023-10-17 2023-10-17 2 3 1990.00 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 +2023-12-19 2023-12-19 2 3 1592.00 -- !query_3_0_after -- -2023-10-17 2023-10-17 2 3 1990.00 +2023-10-17 2023-10-17 2 3 1592.00 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 +2023-12-19 2023-12-19 2 3 1592.00 -- !query_4_0_before -- 2023-10-18 2023-10-18 2 3 1747.20 @@ -22,12 +24,13 @@ 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 -- !query_7_0_after -- 2023-10-17 2023-10-17 2 3 1990.00 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 -2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 -- !query_8_0_before -- 2023-10-18 2023-10-18 2 3 1747.20 @@ -41,11 +44,14 @@ 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 -- !query_11_0_after -- +2023-10-17 2023-10-17 2 3 1990.00 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 \N 2 3 \N +2023-12-19 2023-12-19 2 3 1592.00 -- !query_12_0_before -- 2023-10-18 2023-10-18 2 3 1747.20 @@ -64,16 +70,17 @@ 2023-10-19 2023-10-19 2 3 398.00 -- !query_17_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 2023-10-21 \N 2 3 \N -2023-11-21 2023-11-21 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 -- !query_17_0_after -- 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 2023-10-21 \N 2 3 \N -2023-11-21 2023-11-21 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 -- !query_18_0_before -- 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 @@ -84,18 +91,20 @@ 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 -- !query_19_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 2023-10-21 \N 2 3 \N 2023-11-21 2023-11-21 \N 2 3 \N 2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 -- !query_19_0_after -- +2023-09-17 2023-09-17 \N 2 3 \N 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 2023-10-21 2023-10-21 \N 2 3 \N -2023-11-21 2023-11-21 \N 2 3 \N -2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 -- !query_20_0_before -- 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 @@ -105,3 +114,28 @@ 2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +-- !query_21_0_before -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_21_0_after -- +2023-09-17 2023-09-17 \N 2 3 \N +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 +2023-10-21 2023-10-21 \N 2 3 \N +2023-11-21 2023-11-21 \N 2 3 \N +2023-11-22 2023-11-22 \N 2 3 \N +2023-12-19 2023-12-19 2023-12-19 2 3 1592.00 + +-- !query_22_0_before -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + +-- !query_22_0_after -- +2023-10-18 2023-10-18 2023-10-18 2 3 1747.20 +2023-10-19 2023-10-19 2023-10-19 2 3 1592.00 + diff --git a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy index b9e671f2636b82..d4e3a80dbeb481 100644 --- a/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/availability/grace_period.groovy @@ -149,7 +149,7 @@ suite("grace_period") { """ // force consistency when partition table, and query use the partition changed, should fail - mv_rewrite_fail(""" + mv_not_part_in(""" select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total from lineitem_partition @@ -310,7 +310,7 @@ suite("grace_period") { sql "SET enable_materialized_view_rewrite=true" Thread.sleep(15000); // after 10s when partition table, and query use the partition changed, should fail - mv_rewrite_fail( + mv_not_part_in( """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total diff --git a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.groovy similarity index 68% rename from regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy rename to regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.groovy index eba1edd94b6c3e..0126d5993bbb2d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/partition_mv_rewrite.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/partition_mv_rewrite.groovy @@ -1,3 +1,5 @@ +package mv.partition_union_rewrite + import java.text.SimpleDateFormat // Licensed to the Apache Software Foundation (ASF) under one @@ -40,7 +42,7 @@ suite("partition_mv_rewrite") { ) DUPLICATE KEY(o_orderkey, o_custkey) PARTITION BY RANGE(o_orderdate)( - FROM ('2023-10-16') TO ('2023-11-30') INTERVAL 1 DAY + FROM ('2023-09-16') TO ('2023-12-30') INTERVAL 1 DAY ) DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 PROPERTIES ( @@ -74,7 +76,7 @@ suite("partition_mv_rewrite") { ) DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) PARTITION BY RANGE(l_shipdate) - (FROM ('2023-10-16') TO ('2023-11-30') INTERVAL 1 DAY) + (FROM ('2023-09-16') TO ('2023-12-30') INTERVAL 1 DAY) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 PROPERTIES ( "replication_num" = "1" @@ -94,7 +96,11 @@ suite("partition_mv_rewrite") { (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), - (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'); + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'); """ sql """ @@ -110,7 +116,11 @@ suite("partition_mv_rewrite") { (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'), (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'), (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'), - (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'); + (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'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'); """ @@ -126,7 +136,7 @@ suite("partition_mv_rewrite") { l_suppkey; """ - def all_partition_sql = """ + def query_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 @@ -138,7 +148,7 @@ suite("partition_mv_rewrite") { """ - def partition_sql = """ + def query_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 @@ -151,10 +161,21 @@ suite("partition_mv_rewrite") { """ - sql """DROP MATERIALIZED VIEW IF EXISTS mv_10086""" - sql """DROP TABLE IF EXISTS mv_10086""" + multi_sql """ + analyze table lineitem with sync; + analyze table orders with sync; + """ + + sql """alter table orders modify column o_comment set stats ('row_count'='20');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='20');""" + + + def mv_1_partition_name = "mv_10086" + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" + sql """DROP TABLE IF EXISTS ${mv_1_partition_name}""" sql""" - CREATE MATERIALIZED VIEW mv_10086 + CREATE MATERIALIZED VIEW ${mv_1_partition_name} BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by(l_shipdate) DISTRIBUTED BY RANDOM BUCKETS 2 @@ -163,98 +184,96 @@ suite("partition_mv_rewrite") { ${mv_def_sql} """ - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) multi_sql """ - analyze table lineitem with sync; - analyze table orders with sync; - analyze table mv_10086 with sync; + analyze table ${mv_1_partition_name} with sync; """ - sleep(10000) - mv_rewrite_success(all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - mv_rewrite_success(partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - // base table partition data change + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + // test base table partition data change 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'); """ - waitingPartitionIsExpected("mv_10086", "p_20231017_20231018", false) + waitingPartitionIsExpected(mv_1_partition_name, "p_20231017_20231018", false) + - // enable union rewrite sql "SET enable_materialized_view_rewrite=false" - order_qt_query_3_0_before "${all_partition_sql}" + order_qt_query_3_0_before "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=true" - sql "analyze table mv_10086 with sync" - def memo = sql "explain memo plan ${all_partition_sql}" - print(memo) + + // should rewrite successful when union rewrite enalbe if sub partition is invalid - mv_rewrite_success(all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_3_0_after "${all_partition_sql}" + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_3_0_after "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=false" - order_qt_query_4_0_before "${partition_sql}" + order_qt_query_4_0_before "${query_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successfully when union rewrite enable if doesn't query invalid partition - mv_rewrite_success(partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_4_0_after "${partition_sql}" + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_4_0_after "${query_partition_sql}" // base table add partition - sql "REFRESH MATERIALIZED VIEW mv_10086 AUTO" - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) sql """ insert into lineitem values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-21', '2023-10-21', '2023-10-21', 'a', 'b', 'yyyyyyyyy'); """ - waitingPartitionIsExpected("mv_10086", "p_20231021_20231022", false) + waitingPartitionIsExpected(mv_1_partition_name, "p_20231021_20231022", false) + - // enable union rewrite sql "SET enable_materialized_view_rewrite=false" - order_qt_query_7_0_before "${all_partition_sql}" + order_qt_query_7_0_before "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successful when union rewrite enalbe if base table add new partition - mv_rewrite_success(all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_7_0_after "${all_partition_sql}" + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_7_0_after "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=false" - order_qt_query_8_0_before "${partition_sql}" + order_qt_query_8_0_before "${query_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successfully when union rewrite enable if doesn't query new partition - mv_rewrite_success(partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_8_0_after "${partition_sql}" + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_8_0_after "${query_partition_sql}" // base table delete partition test - sql "REFRESH MATERIALIZED VIEW mv_10086 AUTO" - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) sql """ ALTER TABLE lineitem DROP PARTITION IF EXISTS p_20231017 FORCE; """ // show partitions will cause error, tmp comment - waitingPartitionIsExpected("mv_10086", "p_20231017_20231018", false) + waitingPartitionIsExpected(mv_1_partition_name, "p_20231017_20231018", false) + - // enable union rewrite sql "SET enable_materialized_view_rewrite=false" - order_qt_query_11_0_before "${all_partition_sql}" + order_qt_query_11_0_before "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successful when union rewrite enalbe if base table delete partition - mv_rewrite_success(all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_11_0_after "${all_partition_sql}" + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_11_0_after "${query_all_partition_sql}" sql "SET enable_materialized_view_rewrite=false" - order_qt_query_12_0_before "${partition_sql}" + order_qt_query_12_0_before "${query_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successfully when union rewrite enable if doesn't query deleted partition - mv_rewrite_success(partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_12_0_after "${partition_sql}" - sql """ DROP MATERIALIZED VIEW IF EXISTS mv_10086""" + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_12_0_after "${query_partition_sql}" + sql """ DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" // test mv with ttl def today_str = new SimpleDateFormat("yyyy-MM-dd").format(new Date()).toString(); @@ -285,7 +304,7 @@ suite("partition_mv_rewrite") { PARTITION BY RANGE(l_shipdate) ( PARTITION `p1` VALUES LESS THAN ("2023-10-18"), - PARTITION `p2` VALUES [("2023-10-18"), ("2023-10-20")), + PARTITION `p2` VALUES [("2023-10-18"), ("2023-12-20")), PARTITION `other` VALUES LESS THAN (MAXVALUE) ) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 @@ -297,13 +316,20 @@ suite("partition_mv_rewrite") { insert into lineitem_static 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'); + (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'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'); """ sql """ insert into lineitem_static values (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '${today_str}', '${today_str}', '${today_str}', 'a', 'b', 'yyyyyyyyy'); """ + multi_sql """ + analyze table lineitem_static with sync; + """ + sql """alter table lineitem_static modify column l_comment set stats ('row_count'='6');""" + + def ttl_mv_def_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total @@ -315,7 +341,7 @@ suite("partition_mv_rewrite") { l_partkey, l_suppkey; """ - def ttl_all_partition_sql = """ + def query_ttl_all_partition_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total from lineitem_static left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate @@ -325,7 +351,7 @@ suite("partition_mv_rewrite") { l_partkey, l_suppkey; """ - def ttl_partition_sql = """ + def query_ttl_partition_sql = """ select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total from lineitem_static left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate @@ -357,30 +383,24 @@ suite("partition_mv_rewrite") { create_ttl_mtmv(db, ttl_mv_name, ttl_mv_def_sql) - multi_sql """ - analyze table lineitem_static with sync; - analyze table lineitem with sync; - analyze table orders with sync; - """ - // test when mv is ttl - // enable union rewrite + // test when mv is partition roll up sql "SET enable_materialized_view_rewrite=true" // should rewrite successful when union rewrite enalbe and mv is ttl, query the partition which is in mv - mv_rewrite_success(ttl_all_partition_sql, ttl_mv_name, true, + mv_rewrite_success(query_ttl_all_partition_sql, ttl_mv_name, true, is_partition_statistics_ready(db, ["lineitem_static", "orders", ttl_mv_name])) sql "SET enable_materialized_view_rewrite=false" - order_qt_query_16_0_before "${ttl_partition_sql}" + order_qt_query_16_0_before "${query_ttl_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite fail when union rewrite enalbe and query the partition which is not in mv - mv_rewrite_fail(ttl_partition_sql, ttl_mv_name) - order_qt_query_16_0_after "${ttl_partition_sql}" + mv_not_part_in(query_ttl_partition_sql, ttl_mv_name) + order_qt_query_16_0_after "${query_ttl_partition_sql}" sql """ DROP MATERIALIZED VIEW IF EXISTS ${ttl_mv_name}""" - // date roll up mv + // test date roll up mv partition rewrite def roll_up_mv_def_sql = """ select date_trunc(`l_shipdate`, 'day') as col1, l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total @@ -394,7 +414,7 @@ suite("partition_mv_rewrite") { l_suppkey; """ - def roll_up_all_partition_sql = """ + def query_roll_up_all_partition_sql = """ select date_trunc(`l_shipdate`, 'day') as col1, l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total from lineitem @@ -407,7 +427,7 @@ suite("partition_mv_rewrite") { l_suppkey; """ - def roll_up_partition_sql = """ + def query_roll_up_partition_sql = """ select date_trunc(`l_shipdate`, 'day') as col1, l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total from lineitem @@ -421,10 +441,10 @@ suite("partition_mv_rewrite") { l_suppkey; """ - sql """DROP MATERIALIZED VIEW IF EXISTS mv_10086""" - sql """DROP TABLE IF EXISTS mv_10086""" + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" + sql """DROP TABLE IF EXISTS ${mv_1_partition_name}""" sql""" - CREATE MATERIALIZED VIEW mv_10086 + CREATE MATERIALIZED VIEW ${mv_1_partition_name} BUILD IMMEDIATE REFRESH AUTO ON MANUAL partition by (date_trunc(`col1`, 'month')) DISTRIBUTED BY RANDOM BUCKETS 2 @@ -432,53 +452,43 @@ suite("partition_mv_rewrite") { AS ${roll_up_mv_def_sql} """ - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) - multi_sql """ - analyze table lineitem_static with sync; - analyze table lineitem with sync; - analyze table orders with sync; - """ + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) - mv_rewrite_success(roll_up_all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - mv_rewrite_success(roll_up_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) // base table add partition sql """ insert into lineitem values - (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-11-21', '2023-11-21', '2023-11-21', 'a', 'b', 'yyyyyyyyy'); + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-09-17', '2023-09-17', '2023-09-17', 'a', 'b', 'yyyyyyyyy'); """ - // enable union rewrite + sql "SET enable_materialized_view_rewrite=false" - order_qt_query_17_0_before "${roll_up_all_partition_sql}" + order_qt_query_17_0_before "${query_roll_up_all_partition_sql}" sql "SET enable_materialized_view_rewrite=true" - - multi_sql """ - analyze table lineitem_static with sync; - analyze table lineitem with sync; - analyze table orders with sync; - """ + // should rewrite successful when union rewrite enalbe if base table add new partition - mv_rewrite_success(roll_up_all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_17_0_after "${roll_up_all_partition_sql}" + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_17_0_after "${query_roll_up_all_partition_sql}" sql "SET enable_materialized_view_rewrite=false" - order_qt_query_18_0_before "${roll_up_partition_sql}" + order_qt_query_18_0_before "${query_roll_up_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successfully when union rewrite enable if doesn't query new partition - mv_rewrite_success(roll_up_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_18_0_after "${roll_up_partition_sql}" + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_18_0_after "${query_roll_up_partition_sql}" - // base table partition add data - sql "REFRESH MATERIALIZED VIEW mv_10086 AUTO" - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + // base table partition modify data + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) sql """ insert into lineitem values @@ -486,62 +496,45 @@ suite("partition_mv_rewrite") { (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-11-22', '2023-11-22', '2023-11-22', 'd', 'd', 'yyyyyyyyy'); """ - // enable union rewrite + sql "SET enable_materialized_view_rewrite=false" - order_qt_query_19_0_before "${roll_up_all_partition_sql}" + order_qt_query_19_0_before "${query_roll_up_all_partition_sql}" sql "SET enable_materialized_view_rewrite=true" - - multi_sql """ - analyze table lineitem_static with sync; - analyze table lineitem with sync; - analyze table orders with sync; - """ - - sql """alter table orders modify column o_comment set stats ('row_count'='3');""" - sql """alter table lineitem modify column l_comment set stats ('row_count'='6');""" - sql """alter table lineitem_static modify column l_comment set stats ('row_count'='4');""" // should rewrite successful when union rewrite enalbe if base table add new partition - mv_rewrite_success(roll_up_all_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) - order_qt_query_19_0_after "${roll_up_all_partition_sql}" + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_19_0_after "${query_roll_up_all_partition_sql}" sql "SET enable_materialized_view_rewrite=false" - order_qt_query_20_0_before "${roll_up_partition_sql}" + order_qt_query_20_0_before "${query_roll_up_partition_sql}" sql "SET enable_materialized_view_rewrite=true" // should rewrite successfully when union rewrite enable if doesn't query new partition - mv_rewrite_success(roll_up_partition_sql, "mv_10086", true, - is_partition_statistics_ready(db, ["lineitem", "orders", "mv_10086"])) + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) - order_qt_query_20_0_after "${roll_up_partition_sql}" + order_qt_query_20_0_after "${query_roll_up_partition_sql}" - // base table delete partition - sql "REFRESH MATERIALIZED VIEW mv_10086 AUTO" - waitingMTMVTaskFinished(getJobName(db, "mv_10086")) + // test base table delete partition + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) sql """ ALTER TABLE lineitem DROP PARTITION IF EXISTS p_20231121 FORCE; """ - // enable union rewrite -// this depends on getting corret partitions when base table delete partition, tmp comment -// sql "SET enable_materialized_view_rewrite=false" -// order_qt_query_21_0_before "${roll_up_all_partition_sql}" -// sql "SET enable_materialized_view_rewrite=true" -// explain { -// sql("${roll_up_all_partition_sql}") -// // should rewrite successful when union rewrite enalbe if base table add new partition -// contains("mv_10086(mv_10086)") -// } -// order_qt_query_21_0_after "${roll_up_all_partition_sql}" -// -// sql "SET enable_materialized_view_rewrite=false" -// order_qt_query_22_0_before "${roll_up_partition_sql}" -// sql "SET enable_materialized_view_rewrite=true" -// explain { -// sql("${roll_up_partition_sql}") -// // should rewrite successfully when union rewrite enable if doesn't query new partition -// contains("mv_10086(mv_10086)") -// } -// order_qt_query_22_0_after "${roll_up_partition_sql}" + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_21_0_before "${query_roll_up_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_21_0_after "${query_roll_up_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_22_0_before "${query_roll_up_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + order_qt_query_22_0_after "${query_roll_up_partition_sql}" } diff --git a/regression-test/suites/nereids_rules_p0/mv/rewrite_duration_exceeded/rewrite_duration_exceeded.groovy b/regression-test/suites/nereids_rules_p0/mv/rewrite_duration_exceeded/rewrite_duration_exceeded.groovy new file mode 100644 index 00000000000000..283e5336ec2a39 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/rewrite_duration_exceeded/rewrite_duration_exceeded.groovy @@ -0,0 +1,156 @@ +// 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("rewrite_duration_exceeded") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "SET enable_agg_state = true" + + 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, + public_col INT 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, + public_col INT 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, + public_col INT 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', 1), + (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', null), + (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', 2), + (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', null), + (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', 3); + """ + + sql """ + insert into orders values + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy', 1), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy', null), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy', 2), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy', null), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy', 3), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm', null), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi', 4), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi', null); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1', 1), + (2, 3, 10, 11.01, 'supply2', null); + """ + create_async_mv(db, "mv_1", """ + 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), + 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; + """) + + sql """set materialized_view_rewrite_duration_threshold_ms = -1;""" + + // should materialized view rewrite duration is exceeded + explain { + sql(""" 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), + 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; + """) + check { result -> + contains("materialized view rewrite duration is exceeded") + } + } +} diff --git a/regression-test/suites/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.groovy b/regression-test/suites/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.groovy new file mode 100644 index 00000000000000..55394292d0b75c --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/union_rewrite_grace_big/unioin_rewrite_grace_big.groovy @@ -0,0 +1,549 @@ +package mv.partition_union_rewrite + +import java.text.SimpleDateFormat + +// 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("union_rewrite_grace_big") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF" + + 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(9) 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-09-16') TO ('2023-12-30') INTERVAL 1 DAY + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem + """ + + // test pre init partition + 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-09-16') TO ('2023-12-30') 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'), + (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), + (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), + (1, 1, 'ok', 99.5, '2023-10-17', 'a', 'b', 1, 'yy'), + (2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'), + (2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'), + (2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'), + (2, 2, 'ok', 109.2, '2023-10-18', 'c','d',2, 'mm'), + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 99.5, '2023-12-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'), + (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'), + (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'), + (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'), + (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'), + (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'), + (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'), + (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'), + (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'), + (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'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-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 query_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 query_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; + """ + + + multi_sql """ + analyze table lineitem with sync; + analyze table orders with sync; + """ + + sql """alter table orders modify column o_comment set stats ('row_count'='20');""" + sql """alter table lineitem modify column l_comment set stats ('row_count'='20');""" + + + def mv_1_partition_name = "mv_10086" + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" + sql """DROP TABLE IF EXISTS ${mv_1_partition_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_1_partition_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1', + 'grace_period' = '31536000') + AS + ${mv_def_sql} + """ + + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + + multi_sql """ + analyze table ${mv_1_partition_name} with sync; + """ + + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + // test base table partition data change + 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'); + """ + waitingPartitionIsExpected(mv_1_partition_name, "p_20231017_20231018", false) + + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_3_0_before "${query_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + + + // should rewrite successful when union rewrite enalbe if sub partition is invalid + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_3_0_after "${query_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_4_0_before "${query_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successfully when union rewrite enable if doesn't query invalid partition + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_4_0_after "${query_partition_sql}" + + // base table add partition + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-10-21', '2023-10-21', '2023-10-21', 'a', 'b', 'yyyyyyyyy'); + """ + + waitingPartitionIsExpected(mv_1_partition_name, "p_20231021_20231022", false) + + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_7_0_before "${query_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successful when union rewrite enalbe if base table add new partition + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_7_0_after "${query_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_8_0_before "${query_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successfully when union rewrite enable if doesn't query new partition + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_8_0_after "${query_partition_sql}" + + // base table delete partition test + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + sql """ ALTER TABLE lineitem DROP PARTITION IF EXISTS p_20231017 FORCE; + """ + // show partitions will cause error, tmp comment + waitingPartitionIsExpected(mv_1_partition_name, "p_20231017_20231018", false) + + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_11_0_before "${query_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successful when union rewrite enalbe if base table delete partition + mv_rewrite_success(query_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_11_0_after "${query_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_12_0_before "${query_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successfully when union rewrite enable if doesn't query deleted partition + mv_rewrite_success(query_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_12_0_after "${query_partition_sql}" + sql """ DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" + + // test mv with ttl + def today_str = new SimpleDateFormat("yyyy-MM-dd").format(new Date()).toString(); + + sql """ + drop table if exists lineitem_static; + """ + sql""" + CREATE TABLE IF NOT EXISTS lineitem_static ( + 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 `p1` VALUES LESS THAN ("2023-10-18"), + PARTITION `p2` VALUES [("2023-10-18"), ("2023-12-20")), + PARTITION `other` VALUES LESS THAN (MAXVALUE) + ) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into lineitem_static 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'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-19', '2023-12-19', '2023-12-19', 'c', 'd', 'xxxxxxxxx'); + """ + sql """ + insert into lineitem_static values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '${today_str}', '${today_str}', '${today_str}', 'a', 'b', 'yyyyyyyyy'); + """ + + multi_sql """ + analyze table lineitem_static with sync; + """ + sql """alter table lineitem_static modify column l_comment set stats ('row_count'='6');""" + + + def ttl_mv_def_sql = """ + select l_shipdate, o_orderdate, l_partkey, + l_suppkey, sum(o_totalprice) as sum_total + from lineitem_static + left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query_ttl_all_partition_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total + from lineitem_static + left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + def query_ttl_partition_sql = """ + select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total + from lineitem_static + left join orders on l_orderkey = 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; + """ + def ttl_mv_name = "mv_10000" + + def create_ttl_mtmv = { db_name, mv_inner_name, mv_inner_sql -> + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_inner_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_inner_name} + BUILD IMMEDIATE REFRESH COMPLETE ON MANUAL + PARTITION BY(l_shipdate) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'grace_period' = '31536000', + 'replication_num' = '1', + 'partition_sync_limit' = 2, + 'partition_sync_time_unit' = 'DAY', + 'partition_date_format' = 'yyyy-MM-dd') + AS ${mv_inner_sql} + """ + waitingMTMVTaskFinished(getJobName(db_name, mv_inner_name)) + } + + create_ttl_mtmv(db, ttl_mv_name, ttl_mv_def_sql) + + + // test when mv is partition roll up + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successful when union rewrite enalbe and mv is ttl, query the partition which is in mv + mv_rewrite_success(query_ttl_all_partition_sql, ttl_mv_name, true, + is_partition_statistics_ready(db, ["lineitem_static", "orders", ttl_mv_name])) + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_16_0_before "${query_ttl_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite fail when union rewrite enalbe and query the partition which is not in mv + mv_rewrite_fail(query_ttl_partition_sql, ttl_mv_name) + order_qt_query_16_0_after "${query_ttl_partition_sql}" + + sql """ DROP MATERIALIZED VIEW IF EXISTS ${ttl_mv_name}""" + + + // test date roll up mv partition rewrite + def roll_up_mv_def_sql = """ + select date_trunc(`l_shipdate`, 'day') as col1, 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 + col1, + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + + def query_roll_up_all_partition_sql = """ + select date_trunc(`l_shipdate`, 'day') as col1, 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 + col1, + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + + def query_roll_up_partition_sql = """ + select date_trunc(`l_shipdate`, 'day') as col1, 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 + col1, + l_shipdate, + o_orderdate, + l_partkey, + l_suppkey; + """ + + sql """DROP MATERIALIZED VIEW IF EXISTS ${mv_1_partition_name}""" + sql """DROP TABLE IF EXISTS ${mv_1_partition_name}""" + sql""" + CREATE MATERIALIZED VIEW ${mv_1_partition_name} + BUILD IMMEDIATE REFRESH AUTO ON MANUAL + partition by (date_trunc(`col1`, 'month')) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'grace_period' = '31536000', + 'replication_num' = '1' + ) + AS + ${roll_up_mv_def_sql} + """ + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + + + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + // base table add partition + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-09-17', '2023-09-17', '2023-09-17', 'a', 'b', 'yyyyyyyyy'); + """ + + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_17_0_before "${query_roll_up_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + + + // should rewrite successful when union rewrite enalbe if base table add new partition + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_17_0_after "${query_roll_up_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_18_0_before "${query_roll_up_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successfully when union rewrite enable if doesn't query new partition + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_18_0_after "${query_roll_up_partition_sql}" + + + // base table partition modify data + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + + sql """ + insert into lineitem values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-11-21', '2023-11-21', '2023-11-21', 'd', 'd', 'yyyyyyyyy'), + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-11-22', '2023-11-22', '2023-11-22', 'd', 'd', 'yyyyyyyyy'); + """ + + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_19_0_before "${query_roll_up_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + + + // should rewrite successful when union rewrite enalbe if base table add new partition + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_19_0_after "${query_roll_up_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_20_0_before "${query_roll_up_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + // should rewrite successfully when union rewrite enable if doesn't query new partition + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + order_qt_query_20_0_after "${query_roll_up_partition_sql}" + + + // test base table delete partition + sql "REFRESH MATERIALIZED VIEW ${mv_1_partition_name} AUTO" + waitingMTMVTaskFinished(getJobName(db, mv_1_partition_name)) + sql """ ALTER TABLE lineitem DROP PARTITION IF EXISTS p_20231121 FORCE; + """ + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_21_0_before "${query_roll_up_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + mv_rewrite_success(query_roll_up_all_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + order_qt_query_21_0_after "${query_roll_up_all_partition_sql}" + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_22_0_before "${query_roll_up_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + mv_rewrite_success(query_roll_up_partition_sql, mv_1_partition_name, true, + is_partition_statistics_ready(db, ["lineitem", "orders", mv_1_partition_name])) + + order_qt_query_22_0_after "${query_roll_up_partition_sql}" +} + + +