@ -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 @@ public class SummaryProfile {
|
||||
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 class SummaryProfile {
|
||||
this.nereidsLockTableFinishTime = TimeUtils.getStartTimeMs();
|
||||
}
|
||||
|
||||
public void setNereidsCollectTablePartitionFinishTime() {
|
||||
this.nereidsCollectTablePartitionFinishTime = TimeUtils.getStartTimeMs();
|
||||
}
|
||||
|
||||
public void setNereidsAnalysisTime() {
|
||||
this.nereidsAnalysisFinishTime = TimeUtils.getStartTimeMs();
|
||||
}
|
||||
@ -663,8 +673,13 @@ public class SummaryProfile {
|
||||
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() {
|
||||
|
||||
@ -19,6 +19,7 @@ package org.apache.doris.mtmv;
|
||||
|
||||
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.common.TaskStatus;
|
||||
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.commons.collections.CollectionUtils;
|
||||
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 class MTMVRelationManager implements MTMVHookService {
|
||||
boolean forceConsistent, BiPredicate<ConnectContext, MTMV> predicate) {
|
||||
Set<MTMV> res = Sets.newLinkedHashSet();
|
||||
Set<BaseTableInfo> mvInfos = getMTMVInfos(tableInfos);
|
||||
Map<List<String>, Set<String>> 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 class MTMVRelationManager implements MTMVHookService {
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent,
|
||||
Set<String> relatedPartitions) {
|
||||
long currentTimeMillis = System.currentTimeMillis();
|
||||
return !CollectionUtils
|
||||
.isEmpty(MTMVRewriteUtil.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMillis, forceConsistent));
|
||||
Collection<Partition> 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<BaseTableInfo> getMTMVInfos(List<BaseTableInfo> tableInfos) {
|
||||
|
||||
@ -25,12 +25,16 @@ import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState;
|
||||
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<Partition> getMTMVCanRewritePartitions(MTMV mtmv, ConnectContext ctx,
|
||||
long currentTimeMills, boolean forceConsistent) {
|
||||
long currentTimeMills, boolean forceConsistent,
|
||||
Set<String> relatedPartitions) {
|
||||
List<Partition> res = Lists.newArrayList();
|
||||
Collection<Partition> allPartitions = mtmv.getPartitions();
|
||||
MTMVRelation mtmvRelation = mtmv.getRelation();
|
||||
@ -55,6 +60,11 @@ public class MTMVRewriteUtil {
|
||||
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<String> mtmvNeedComparePartitions = null;
|
||||
MTMVRefreshContext refreshContext = null;
|
||||
// check gracePeriod
|
||||
long gracePeriodMills = mtmv.getGracePeriod();
|
||||
@ -73,6 +83,14 @@ public class MTMVRewriteUtil {
|
||||
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 class MTMVRewriteUtil {
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
private static Set<String> getMtmvPartitionsByRelatedPartitions(MTMV mtmv, MTMVRefreshContext refreshContext,
|
||||
Set<String> 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<String> res = Sets.newHashSet();
|
||||
Map<String, String> relatedToMv = getRelatedToMv(refreshContext.getPartitionMappings());
|
||||
for (String relatedPartition : relatedPartitions) {
|
||||
res.add(relatedToMv.get(relatedPartition));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
private static Map<String, String> getRelatedToMv(Map<String, Set<String>> mvToRelated) {
|
||||
Map<String, String> res = Maps.newHashMap();
|
||||
for (Entry<String, Set<String>> entry : mvToRelated.entrySet()) {
|
||||
for (String relatedPartition : entry.getValue()) {
|
||||
res.put(relatedPartition, entry.getKey());
|
||||
}
|
||||
}
|
||||
return res;
|
||||
}
|
||||
}
|
||||
|
||||
@ -24,6 +24,7 @@ import org.apache.doris.nereids.jobs.Job;
|
||||
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 class CascadesContext implements ScheduleContext {
|
||||
return new TableCollectAndHookInitializer(this);
|
||||
}
|
||||
|
||||
public TablePartitionCollector newTablePartitionCollector() {
|
||||
return new TablePartitionCollector(this);
|
||||
}
|
||||
|
||||
public Analyzer newAnalyzer() {
|
||||
return new Analyzer(this);
|
||||
}
|
||||
|
||||
@ -344,7 +344,22 @@ public class NereidsPlanner extends Planner {
|
||||
}
|
||||
}
|
||||
|
||||
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 @@ public class NereidsPlanner extends Planner {
|
||||
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
|
||||
|
||||
@ -35,4 +35,16 @@ public interface PlannerHook {
|
||||
*/
|
||||
default void afterAnalyze(NereidsPlanner planner) {
|
||||
}
|
||||
|
||||
/**
|
||||
* the hook before rewrite
|
||||
*/
|
||||
default void beforeRewrite(NereidsPlanner planner) {
|
||||
}
|
||||
|
||||
/**
|
||||
* the hook after rewrite
|
||||
*/
|
||||
default void afterRewrite(NereidsPlanner planner) {
|
||||
}
|
||||
}
|
||||
|
||||
@ -19,6 +19,7 @@ package org.apache.doris.nereids;
|
||||
|
||||
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.common.Pair;
|
||||
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.Stopwatch;
|
||||
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 class StatementContext implements Closeable {
|
||||
private ConnectContext connectContext;
|
||||
|
||||
private final Stopwatch stopwatch = Stopwatch.createUnstarted();
|
||||
private final Stopwatch materializedViewStopwatch = Stopwatch.createUnstarted();
|
||||
|
||||
@GuardedBy("this")
|
||||
private final Map<String, Supplier<Object>> contextCacheMap = Maps.newLinkedHashMap();
|
||||
@ -175,7 +179,14 @@ public class StatementContext implements Closeable {
|
||||
|
||||
// tables in this query directly
|
||||
private final Map<List<String>, 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<List<String>, TableIf> mtmvRelatedTables = Maps.newHashMap();
|
||||
// insert into target tables
|
||||
private final Map<List<String>, TableIf> insertTargetTables = Maps.newHashMap();
|
||||
@ -211,6 +222,16 @@ public class StatementContext implements Closeable {
|
||||
|
||||
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<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap =
|
||||
HashMultimap.create();
|
||||
|
||||
// Record mtmv and valid partitions map because this is time-consuming behavior
|
||||
private final Map<BaseTableInfo, Collection<Partition>> mvCanRewritePartitionsMap = new HashMap<>();
|
||||
|
||||
public StatementContext() {
|
||||
this(ConnectContext.get(), null, 0);
|
||||
}
|
||||
@ -339,6 +360,26 @@ public class StatementContext implements Closeable {
|
||||
return stopwatch;
|
||||
}
|
||||
|
||||
public Stopwatch getMaterializedViewStopwatch() {
|
||||
return materializedViewStopwatch;
|
||||
}
|
||||
|
||||
public long getMaterializedViewRewriteDuration() {
|
||||
return materializedViewRewriteDuration;
|
||||
}
|
||||
|
||||
public void addMaterializedViewRewriteDuration(long millisecond) {
|
||||
materializedViewRewriteDuration += millisecond;
|
||||
}
|
||||
|
||||
public Multimap<List<String>, Pair<RelationId, Set<String>>> getTableUsedPartitionNameMap() {
|
||||
return tableUsedPartitionNameMap;
|
||||
}
|
||||
|
||||
public Map<BaseTableInfo, Collection<Partition>> getMvCanRewritePartitionsMap() {
|
||||
return mvCanRewritePartitionsMap;
|
||||
}
|
||||
|
||||
public void setMaxNAryInnerJoin(int maxNAryInnerJoin) {
|
||||
if (maxNAryInnerJoin > this.maxNAryInnerJoin) {
|
||||
this.maxNAryInnerJoin = maxNAryInnerJoin;
|
||||
|
||||
@ -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<RewriteJob> getJobs() {
|
||||
return buildCollectorJobs();
|
||||
}
|
||||
|
||||
private static List<RewriteJob> buildCollectorJobs() {
|
||||
return jobs(
|
||||
custom(RuleType.COLLECT_PARTITIONS, QueryPartitionCollector::new)
|
||||
);
|
||||
}
|
||||
}
|
||||
@ -25,6 +25,8 @@ import org.apache.doris.nereids.trees.plans.Plan;
|
||||
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 @@ import javax.annotation.Nullable;
|
||||
* Representation for group in cascades optimizer.
|
||||
*/
|
||||
public class StructInfoMap {
|
||||
|
||||
public static final Logger LOG = LogManager.getLogger(StructInfoMap.class);
|
||||
private final Map<BitSet, Pair<GroupExpression, List<BitSet>>> groupExpressionMap = new HashMap<>();
|
||||
private final Map<BitSet, StructInfo> infoMap = new HashMap<>();
|
||||
private long refreshVersion = 0;
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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.Plan;
|
||||
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.qe.SessionVariable;
|
||||
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 org.apache.logging.log4j.Logger;
|
||||
|
||||
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<Plan> rewrite(Plan queryPlan, CascadesContext cascadesContext) {
|
||||
List<Plan> 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 abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
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 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
continue;
|
||||
}
|
||||
Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> invalidPartitions;
|
||||
if (materializationContext instanceof AsyncMaterializationContext) {
|
||||
if (PartitionCompensator.needUnionRewrite(materializationContext)
|
||||
&& sessionVariable.isEnableMaterializedViewUnionRewrite()) {
|
||||
MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv();
|
||||
BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo();
|
||||
Map<List<String>, Set<String>> queryUsedPartitions = PartitionCompensator.getQueryUsedPartitions(
|
||||
cascadesContext.getConnectContext().getStatementContext());
|
||||
Set<String> 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<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> finalInvalidPartitions =
|
||||
invalidPartitions;
|
||||
if (partitionNeedUnion) {
|
||||
MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv();
|
||||
Pair<Plan, Boolean> planAndNeedAddFilterPair =
|
||||
StructInfo.addFilterOnTableScan(queryPlan, invalidPartitions.value(),
|
||||
mtmv.getMvPartitionInfo().getRelatedCol(), cascadesContext);
|
||||
@ -384,13 +431,6 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
}
|
||||
}
|
||||
|
||||
protected boolean needUnionRewrite(
|
||||
Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> 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 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
* @return the key in pair is mvNeedRemovePartitionNameSet, the value in pair is baseTableNeedUnionPartitionNameSet
|
||||
*/
|
||||
protected Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> calcInvalidPartitions(
|
||||
Plan queryPlan, Plan rewrittenPlan,
|
||||
AsyncMaterializationContext materializationContext, CascadesContext cascadesContext)
|
||||
Set<String> queryUsedPartition,
|
||||
Plan rewrittenPlan,
|
||||
CascadesContext cascadesContext,
|
||||
AsyncMaterializationContext materializationContext)
|
||||
throws AnalysisException {
|
||||
Set<String> mvNeedRemovePartitionNameSet = new HashSet<>();
|
||||
Set<String> 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<BaseTableInfo, Set<Partition>> 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<String> queryUsedBaseTablePartitionNameSet = queryUsedBaseTablePartitions.get(relatedPartitionTable)
|
||||
.stream()
|
||||
.map(Partition::getName)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
Collection<Partition> mvValidPartitions = MTMVRewriteUtil.getMTMVCanRewritePartitions(mtmv,
|
||||
cascadesContext.getConnectContext(), System.currentTimeMillis(), false);
|
||||
Set<String> mvValidPartitionNameSet = new HashSet<>();
|
||||
Set<String> mvValidBaseTablePartitionNameSet = new HashSet<>();
|
||||
Set<String> mvValidHasDataRelatedBaseTableNameSet = new HashSet<>();
|
||||
Pair<Map<String, Set<String>>, Map<String, String>> partitionMapping = mtmv.calculateDoublyPartitionMappings();
|
||||
for (Partition mvValidPartition : mvValidPartitions) {
|
||||
mvValidPartitionNameSet.add(mvValidPartition.getName());
|
||||
Set<String> 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<String> rewrittenPlanUsePartitionNameSet = new HashSet<>();
|
||||
List<Object> 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<BaseTableInfo, Set<String>> mvPartitionNeedRemoveNameMap = new HashMap<>();
|
||||
if (!mvNeedRemovePartitionNameSet.isEmpty()) {
|
||||
mvPartitionNeedRemoveNameMap.put(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet);
|
||||
}
|
||||
Map<BaseTableInfo, Set<String>> 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 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
// 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 @@ public abstract class AbstractMaterializedViewRule implements ExplorationRuleFac
|
||||
}
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
@ -406,13 +406,20 @@ public class HyperGraphComparator {
|
||||
}
|
||||
|
||||
private Map<Edge, Edge> constructQueryToViewJoinMapWithExpr() {
|
||||
Map<Expression, Edge> viewExprToEdge = getViewJoinEdges().stream()
|
||||
.flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e)))
|
||||
.collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second));
|
||||
Map<Expression, Edge> queryExprToEdge = getQueryJoinEdges().stream()
|
||||
.flatMap(e -> e.getExpressions().stream().map(expr -> Pair.of(expr, e)))
|
||||
.collect(ImmutableMap.toImmutableMap(p -> p.first, p -> p.second));
|
||||
|
||||
Map<Expression, Edge> viewExprToEdge = new HashMap<>();
|
||||
List<JoinEdge> viewJoinEdges = getViewJoinEdges();
|
||||
for (JoinEdge viewJoin : viewJoinEdges) {
|
||||
for (Expression expression : viewJoin.getExpressions()) {
|
||||
viewExprToEdge.put(expression, viewJoin);
|
||||
}
|
||||
}
|
||||
Map<Expression, Edge> queryExprToEdge = new HashMap<>();
|
||||
List<JoinEdge> queryJoinEdges = getQueryJoinEdges();
|
||||
for (JoinEdge queryJoin : queryJoinEdges) {
|
||||
for (Expression expression : queryJoin.getExpressions()) {
|
||||
queryExprToEdge.put(expression, queryJoin);
|
||||
}
|
||||
}
|
||||
HashMap<Edge, Edge> edgeMap = new HashMap<>();
|
||||
for (Entry<Expression, Edge> entry : queryExprToEdge.entrySet()) {
|
||||
if (edgeMap.containsKey(entry.getValue())) {
|
||||
@ -444,15 +451,19 @@ public class HyperGraphComparator {
|
||||
// +--LogicalOlapScan
|
||||
private Map<Edge, Edge> constructQueryToViewFilterMapWithExpr() {
|
||||
Multimap<Expression, Edge> 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<FilterEdge> viewFilterEdges = getViewFilterEdges();
|
||||
for (FilterEdge viewEdge : viewFilterEdges) {
|
||||
for (Expression expression : viewEdge.getExpressions()) {
|
||||
viewExprToEdge.put(expression, viewEdge);
|
||||
}
|
||||
}
|
||||
Multimap<Expression, Edge> 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<FilterEdge> queryFilterEdges = getQueryFilterEdges();
|
||||
for (FilterEdge queryEdge : queryFilterEdges) {
|
||||
for (Expression expression : queryEdge.getExpressions()) {
|
||||
queryExprToEdge.put(expression, queryEdge);
|
||||
}
|
||||
}
|
||||
HashMap<Edge, Edge> queryToViewEdgeMap = new HashMap<>();
|
||||
for (Entry<Expression, Collection<Edge>> entry : queryExprToEdge.asMap().entrySet()) {
|
||||
Expression queryExprViewBased = null;
|
||||
|
||||
@ -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 @@ public class InitMaterializationContextHook implements PlannerHook {
|
||||
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<MaterializationContext> asyncMaterializationContext = new ArrayList<>();
|
||||
@ -113,13 +113,6 @@ public class InitMaterializationContextHook implements PlannerHook {
|
||||
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;
|
||||
}
|
||||
|
||||
@ -24,6 +24,7 @@ import org.apache.doris.cluster.ClusterNamespace;
|
||||
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 abstract class MaterializationContext {
|
||||
}
|
||||
|
||||
/**
|
||||
* Record fail reason when in rewriting
|
||||
* Record fail reason when in rewriting by struct info
|
||||
*/
|
||||
public void recordFailReason(StructInfo structInfo, String summary, Supplier<String> failureReasonSupplier) {
|
||||
// record it's rewritten
|
||||
@ -342,6 +343,24 @@ public abstract class MaterializationContext {
|
||||
Pair.of(summary, this.isEnableRecordFailureDetail() ? failureReasonSupplier.get() : ""));
|
||||
}
|
||||
|
||||
/**
|
||||
* Record fail reason when in rewriting by queryGroupPlan
|
||||
*/
|
||||
public void recordFailReason(Plan queryGroupPlan, String summary, Supplier<String> 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 abstract class MaterializationContext {
|
||||
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<MaterializationContext> 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<String> qualifiers) {
|
||||
return String.join(".", qualifiers);
|
||||
}
|
||||
|
||||
@ -92,11 +92,15 @@ public class MaterializedViewAggregateOnNoneAggregateRule extends AbstractMateri
|
||||
|
||||
@Override
|
||||
protected Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> calcInvalidPartitions(
|
||||
Plan queryPlan, Plan rewrittenPlan, AsyncMaterializationContext materializationContext,
|
||||
CascadesContext cascadesContext) throws AnalysisException {
|
||||
Set<String> queryUsedPartition,
|
||||
Plan rewrittenPlan,
|
||||
CascadesContext cascadesContext,
|
||||
AsyncMaterializationContext materializationContext)
|
||||
throws AnalysisException {
|
||||
Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> 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;
|
||||
}
|
||||
|
||||
@ -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<RelationId, Set<String>> ALL_PARTITIONS = Pair.of(null, null);
|
||||
public static final Collection<Pair<RelationId, Set<String>>> 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<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> calcInvalidPartitions(
|
||||
Set<String> queryUsedBaseTablePartitionNameSet, Plan rewrittenPlan,
|
||||
AsyncMaterializationContext materializationContext, CascadesContext cascadesContext)
|
||||
throws AnalysisException {
|
||||
Set<String> mvNeedRemovePartitionNameSet = new HashSet<>();
|
||||
Set<String> 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<Partition> mvValidPartitions = cascadesContext.getStatementContext()
|
||||
.getMvCanRewritePartitionsMap().get(new BaseTableInfo(mtmv));
|
||||
Set<String> mvValidPartitionNameSet = new HashSet<>();
|
||||
Set<String> mvValidBaseTablePartitionNameSet = new HashSet<>();
|
||||
Set<String> mvValidHasDataRelatedBaseTableNameSet = new HashSet<>();
|
||||
Pair<Map<String, Set<String>>, Map<String, String>> partitionMapping = mtmv.calculateDoublyPartitionMappings();
|
||||
for (Partition mvValidPartition : mvValidPartitions) {
|
||||
mvValidPartitionNameSet.add(mvValidPartition.getName());
|
||||
Set<String> 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<String> rewrittenPlanUsePartitionNameSet = new HashSet<>();
|
||||
List<Object> 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<BaseTableInfo, Set<String>> mvPartitionNeedRemoveNameMap = new HashMap<>();
|
||||
if (!mvNeedRemovePartitionNameSet.isEmpty()) {
|
||||
mvPartitionNeedRemoveNameMap.put(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet);
|
||||
}
|
||||
Map<BaseTableInfo, Set<String>> baseTablePartitionNeedUnionNameMap = new HashMap<>();
|
||||
if (!baseTableNeedUnionPartitionNameSet.isEmpty()) {
|
||||
baseTablePartitionNeedUnionNameMap.put(relatedPartitionTable, baseTableNeedUnionPartitionNameSet);
|
||||
}
|
||||
return Pair.of(mvPartitionNeedRemoveNameMap, baseTablePartitionNeedUnionNameMap);
|
||||
}
|
||||
|
||||
public static boolean needUnionRewrite(
|
||||
Pair<Map<BaseTableInfo, Set<String>>, Map<BaseTableInfo, Set<String>>> 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<List<String>, Set<String>> 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<List<String>, Pair<RelationId, Set<String>>> 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<List<String>, Set<String>> queryUsedRelatedTablePartitionsMap = new HashMap<>();
|
||||
outer:
|
||||
for (Map.Entry<List<String>, TableIf> queryUsedTableEntry : statementContext.getTables().entrySet()) {
|
||||
Set<String> usedPartitionSet = new HashSet<>();
|
||||
Collection<Pair<RelationId, Set<String>>> tableUsedPartitions =
|
||||
tableUsedPartitionNameMap.get(queryUsedTableEntry.getKey());
|
||||
if (!tableUsedPartitions.isEmpty()) {
|
||||
if (ALL_PARTITIONS_LIST.equals(tableUsedPartitions)) {
|
||||
queryUsedRelatedTablePartitionsMap.put(queryUsedTableEntry.getKey(), null);
|
||||
continue;
|
||||
}
|
||||
for (Pair<RelationId, Set<String>> 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;
|
||||
}
|
||||
}
|
||||
@ -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.rules.exploration.mv.Predicates.SplitPredicate;
|
||||
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.algebra.Project;
|
||||
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.LogicalRepeat;
|
||||
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<ExpressionPosition, Map<Expression, Expression>> expressionToShuttledExpressionToMap;
|
||||
|
||||
// Record the exprId and the corresponding expr map, this is used by expression shuttled
|
||||
private final Map<ExprId, Expression> namedExprIdAndExprMapping;
|
||||
private final List<? extends Expression> planOutputShuttledExpressions;
|
||||
|
||||
/**
|
||||
@ -145,7 +136,6 @@ public class StructInfo {
|
||||
Map<ExpressionPosition, Multimap<Expression, Pair<Expression, HyperElement>>>
|
||||
shuttledExpressionsToExpressionsMap,
|
||||
Map<ExpressionPosition, Map<Expression, Expression>> expressionToShuttledExpressionToMap,
|
||||
Map<ExprId, Expression> namedExprIdAndExprMapping,
|
||||
BitSet tableIdSet,
|
||||
SplitPredicate splitPredicate,
|
||||
EquivalenceClass equivalenceClass,
|
||||
@ -164,7 +154,6 @@ public class StructInfo {
|
||||
this.equivalenceClass = equivalenceClass;
|
||||
this.shuttledExpressionsToExpressionsMap = shuttledExpressionsToExpressionsMap;
|
||||
this.expressionToShuttledExpressionToMap = expressionToShuttledExpressionToMap;
|
||||
this.namedExprIdAndExprMapping = namedExprIdAndExprMapping;
|
||||
this.planOutputShuttledExpressions = planOutputShuttledExpressions;
|
||||
}
|
||||
|
||||
@ -175,8 +164,7 @@ public class StructInfo {
|
||||
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 class StructInfo {
|
||||
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 @@ public class StructInfo {
|
||||
Map<ExpressionPosition, Multimap<Expression, Pair<Expression, HyperElement>>>
|
||||
shuttledExpressionsToExpressionsMap,
|
||||
Map<ExpressionPosition, Map<Expression, Expression>> expressionToShuttledExpressionToMap,
|
||||
Map<ExprId, Expression> namedExprIdAndExprMapping,
|
||||
List<CatalogRelation> relations,
|
||||
Map<RelationId, StructInfoNode> relationIdStructInfoNodeMap,
|
||||
BitSet hyperTableBitSet,
|
||||
@ -213,22 +199,16 @@ public class StructInfo {
|
||||
// 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<Expression> replacedExpressions = replaceContext.getReplacedExpressions();
|
||||
List<Expression> nodeExpressions = structInfoNode.getExpressions();
|
||||
if (nodeExpressions != null) {
|
||||
List<? extends Expression> 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 @@ public class StructInfo {
|
||||
List<? extends Expression> 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<Expression> replacedExpressions = replaceContext.getReplacedExpressions();
|
||||
for (int i = 0; i < replacedExpressions.size(); i++) {
|
||||
List<? extends Expression> 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<? extends Expression> 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<? extends Expression> 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 class StructInfo {
|
||||
Map<RelationId, StructInfoNode> relationIdStructInfoNodeMap = new LinkedHashMap<>();
|
||||
Map<ExpressionPosition, Multimap<Expression, Pair<Expression, HyperElement>>>
|
||||
shuttledHashConjunctsToConjunctsMap = new LinkedHashMap<>();
|
||||
Map<ExprId, Expression> namedExprIdAndExprMapping = new LinkedHashMap<>();
|
||||
BitSet tableBitSet = new BitSet();
|
||||
Map<ExpressionPosition, Map<Expression, Expression>> expressionToShuttledExpressionToMap = new HashMap<>();
|
||||
boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap,
|
||||
expressionToShuttledExpressionToMap,
|
||||
namedExprIdAndExprMapping,
|
||||
relationList,
|
||||
relationIdStructInfoNodeMap,
|
||||
tableBitSet,
|
||||
@ -370,7 +338,7 @@ public class StructInfo {
|
||||
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 class StructInfo {
|
||||
return originalPlanId;
|
||||
}
|
||||
|
||||
public Map<ExprId, Expression> getNamedExprIdAndExprMapping() {
|
||||
return namedExprIdAndExprMapping;
|
||||
}
|
||||
|
||||
public BitSet getTableBitSet() {
|
||||
return tableBitSet;
|
||||
}
|
||||
@ -767,35 +731,6 @@ public class StructInfo {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Collect partitions on base table
|
||||
*/
|
||||
public static class QueryScanPartitionsCollector extends DefaultPlanVisitor<Plan,
|
||||
Map<BaseTableInfo, Set<Partition>>> {
|
||||
@Override
|
||||
public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation,
|
||||
Map<BaseTableInfo, Set<Partition>> 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<Partition> 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
|
||||
*
|
||||
|
||||
@ -64,7 +64,6 @@ public class PruneFileScanPartition extends OneRewriteRuleFactory {
|
||||
// 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);
|
||||
|
||||
@ -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<ConnectContext> 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<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap = context.getStatementContext()
|
||||
.getTableUsedPartitionNameMap();
|
||||
Set<String> 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;
|
||||
}
|
||||
}
|
||||
@ -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.GroupPlan;
|
||||
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 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class ExpressionLineageReplacer extends DefaultPlanVisitor<Expression, ExpressionReplaceContext> {
|
||||
|
||||
public static final Logger LOG = LogManager.getLogger(ExpressionLineageReplacer.class);
|
||||
public static final ExpressionLineageReplacer INSTANCE = new ExpressionLineageReplacer();
|
||||
|
||||
@Override
|
||||
@ -63,25 +64,7 @@ public class ExpressionLineageReplacer extends DefaultPlanVisitor<Expression, Ex
|
||||
|
||||
@Override
|
||||
public Expression visitGroupPlan(GroupPlan groupPlan, ExpressionReplaceContext context) {
|
||||
Group group = groupPlan.getGroup();
|
||||
if (group == null) {
|
||||
return visit(groupPlan, context);
|
||||
}
|
||||
Collection<StructInfo> 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<StructInfo> 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;
|
||||
}
|
||||
|
||||
|
||||
@ -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 class SessionVariable implements Serializable, Writable {
|
||||
"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"})
|
||||
|
||||
@ -27,6 +27,7 @@ import org.apache.doris.qe.ConnectContext;
|
||||
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 class MTMVRewriteUtilTest {
|
||||
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 class MTMVRewriteUtilTest {
|
||||
// currentTimeMills is 3, grace period is 2, and partition getVisibleVersionTime is 1
|
||||
// if forceConsistent this should get 0 partitions which mtmv can use.
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, true);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, true, null);
|
||||
Assert.assertEquals(0, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetMTMVCanRewritePartitionsNormal() {
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(1, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -164,7 +174,8 @@ public class MTMVRewriteUtilTest {
|
||||
};
|
||||
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(1, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -185,7 +196,8 @@ public class MTMVRewriteUtilTest {
|
||||
};
|
||||
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(0, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -199,7 +211,8 @@ public class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> 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 class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(0, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -235,7 +249,8 @@ public class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(1, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -253,7 +268,8 @@ public class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> 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 class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(0, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -283,7 +300,8 @@ public class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(1, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
|
||||
@ -297,7 +315,8 @@ public class MTMVRewriteUtilTest {
|
||||
}
|
||||
};
|
||||
Collection<Partition> mtmvCanRewritePartitions = MTMVRewriteUtil
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false);
|
||||
.getMTMVCanRewritePartitions(mtmv, ctx, currentTimeMills, false,
|
||||
null);
|
||||
Assert.assertEquals(0, mtmvCanRewritePartitions.size());
|
||||
}
|
||||
}
|
||||
|
||||
@ -65,7 +65,8 @@ class StructInfoMapTest extends SqlTestBase {
|
||||
Assertions.assertEquals(1, tableMaps.size());
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent,
|
||||
Set<String> relatedPartitions) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -123,7 +124,8 @@ class StructInfoMapTest extends SqlTestBase {
|
||||
Assertions.assertEquals(1, tableMaps.size());
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent,
|
||||
Set<String> relatedPartitions) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
@ -147,7 +149,6 @@ class StructInfoMapTest extends SqlTestBase {
|
||||
.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 @@ class StructInfoMapTest extends SqlTestBase {
|
||||
);
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent,
|
||||
Set<String> relatedPartitions) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
@ -54,7 +54,8 @@ public class IdStatisticsMapTest extends SqlTestBase {
|
||||
};
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid,
|
||||
Set<String> queryUsedRelatedTablePartitionsMap) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
@ -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.Assertions;
|
||||
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 class MtmvCacheNewConnectContextTest extends SqlTestBase {
|
||||
};
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid,
|
||||
Map<BaseTableInfo, Set<String>> queryUsedRelatedTablePartitionsMap) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
@ -31,6 +31,7 @@ import org.junit.jupiter.api.Assertions;
|
||||
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 class MvTableIdIsLongTest extends SqlTestBase {
|
||||
};
|
||||
new MockUp<MTMVRelationManager>() {
|
||||
@Mock
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid) {
|
||||
public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid,
|
||||
Set<String> queryUsedRelatedTablePartitionsMap) {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
|
||||
@ -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<SessionVariable>() {
|
||||
@Mock
|
||||
public BitSet getDisableNereidsRules() {
|
||||
return disableNereidsRules;
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<OlapTable>() {
|
||||
@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<LogicalOlapScan>() {
|
||||
@Mock
|
||||
public List<Long> 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<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap = c1.getStatementContext()
|
||||
.getTableUsedPartitionNameMap();
|
||||
Map<BaseTableInfo, Collection<Partition>> mvCanRewritePartitionsMap = c1.getStatementContext()
|
||||
.getMvCanRewritePartitionsMap();
|
||||
Assertions.assertFalse(tableUsedPartitionNameMap.isEmpty());
|
||||
|
||||
for (Map.Entry<List<String>, Pair<RelationId, Set<String>>> 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<SessionVariable>() {
|
||||
@Mock
|
||||
public BitSet getDisableNereidsRules() {
|
||||
return disableNereidsRules;
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<PartitionPruner>() {
|
||||
@Mock
|
||||
public <K extends Comparable<K>> List<Long> prune(List<Slot> partitionSlots, Expression partitionPredicate,
|
||||
Map<K, PartitionItem> idToPartitions, CascadesContext cascadesContext,
|
||||
PartitionTableType partitionTableType) {
|
||||
return Lists.newArrayList(1L);
|
||||
}
|
||||
};
|
||||
|
||||
new MockUp<OlapTable>() {
|
||||
@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<LogicalOlapScan>() {
|
||||
@Mock
|
||||
public List<Long> 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<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap = c1.getStatementContext()
|
||||
.getTableUsedPartitionNameMap();
|
||||
Map<BaseTableInfo, Collection<Partition>> mvCanRewritePartitionsMap = c1.getStatementContext()
|
||||
.getMvCanRewritePartitionsMap();
|
||||
Assertions.assertFalse(tableUsedPartitionNameMap.isEmpty());
|
||||
|
||||
for (Map.Entry<List<String>, Pair<RelationId, Set<String>>> 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");
|
||||
}
|
||||
}
|
||||
@ -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<List<String>, Set<String>> queryUsedPartitions
|
||||
= PartitionCompensator.getQueryUsedPartitions(
|
||||
nereidsPlanner.getCascadesContext().getStatementContext());
|
||||
|
||||
List<String> itmeQualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "lineitem_list_partition");
|
||||
Set<String> queryTableUsedPartition = queryUsedPartitions.get(itmeQualifier);
|
||||
Assertions.assertEquals(queryTableUsedPartition, ImmutableSet.of("p1", "p2", "p3"));
|
||||
|
||||
List<String> orderQualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "orders_list_partition");
|
||||
Set<String> 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<String> qualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "lineitem_list_partition");
|
||||
|
||||
Multimap<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap
|
||||
= connectContext.getStatementContext().getTableUsedPartitionNameMap();
|
||||
tableUsedPartitionNameMap.put(qualifier, PartitionCompensator.ALL_PARTITIONS);
|
||||
|
||||
Map<List<String>, Set<String>> queryUsedPartitions
|
||||
= PartitionCompensator.getQueryUsedPartitions(
|
||||
nereidsPlanner.getCascadesContext().getStatementContext());
|
||||
Set<String> queryTableUsedPartition = queryUsedPartitions.get(qualifier);
|
||||
// if tableUsedPartitionNameMap contain any PartitionCompensator.ALL_PARTITIONS
|
||||
// consider query all partitions from table
|
||||
Assertions.assertNull(queryTableUsedPartition);
|
||||
|
||||
List<String> orderQualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "orders_list_partition");
|
||||
Set<String> 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<String> qualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "lineitem_list_partition");
|
||||
|
||||
Multimap<List<String>, Pair<RelationId, Set<String>>> tableUsedPartitionNameMap
|
||||
= connectContext.getStatementContext().getTableUsedPartitionNameMap();
|
||||
tableUsedPartitionNameMap.removeAll(qualifier);
|
||||
tableUsedPartitionNameMap.put(qualifier, PartitionCompensator.ALL_PARTITIONS);
|
||||
|
||||
Map<List<String>, Set<String>> queryUsedPartitions
|
||||
= PartitionCompensator.getQueryUsedPartitions(
|
||||
nereidsPlanner.getCascadesContext().getStatementContext());
|
||||
Set<String> queryTableUsedPartition = queryUsedPartitions.get(qualifier);
|
||||
// if tableUsedPartitionNameMap contain only PartitionCompensator.ALL_PARTITIONS
|
||||
// consider query all partitions from table
|
||||
Assertions.assertNull(queryTableUsedPartition);
|
||||
|
||||
List<String> orderQualifier = ImmutableList.of(
|
||||
"internal", "partition_compensate_test", "orders_list_partition");
|
||||
Set<String> orderTableUsedPartition = queryUsedPartitions.get(orderQualifier);
|
||||
Assertions.assertEquals(orderTableUsedPartition, ImmutableSet.of("p1", "p2", "p3", "p4"));
|
||||
});
|
||||
}
|
||||
}
|
||||
@ -121,7 +121,6 @@ public class PlanChecker {
|
||||
public PlanChecker analyze() {
|
||||
this.cascadesContext.newAnalyzer().analyze();
|
||||
this.cascadesContext.toMemo();
|
||||
InitMaterializationContextHook.INSTANCE.initMaterializationContext(this.cascadesContext);
|
||||
return this;
|
||||
}
|
||||
|
||||
@ -245,6 +244,8 @@ public class PlanChecker {
|
||||
|
||||
public PlanChecker rewrite() {
|
||||
Rewriter.getWholeTreeRewriter(cascadesContext).execute();
|
||||
cascadesContext.newTablePartitionCollector().execute();
|
||||
InitMaterializationContextHook.INSTANCE.initMaterializationContext(this.cascadesContext);
|
||||
cascadesContext.toMemo();
|
||||
return this;
|
||||
}
|
||||
@ -553,8 +554,10 @@ public class PlanChecker {
|
||||
|
||||
public PlanChecker checkExplain(String sql, Consumer<NereidsPlanner> 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 class PlanChecker {
|
||||
|
||||
public PlanChecker checkPlannerResult(String sql, Consumer<NereidsPlanner> 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;
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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}"
|
||||
}
|
||||
@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -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}"
|
||||
}
|
||||
|
||||
|
||||
|
||||
Reference in New Issue
Block a user