diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java index d6bce0329c02a1..32cfb3c38625a2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MTMV.java @@ -21,12 +21,12 @@ import org.apache.doris.catalog.OlapTableFactory.MTMVParams; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Config; -import org.apache.doris.common.Pair; import org.apache.doris.common.util.PropertyAnalyzer; import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.info.TableNameInfo; import org.apache.doris.job.common.TaskStatus; import org.apache.doris.job.extensions.mtmv.MTMVTask; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.EnvInfo; import org.apache.doris.mtmv.MTMVCache; import org.apache.doris.mtmv.MTMVJobInfo; @@ -40,17 +40,21 @@ import org.apache.doris.mtmv.MTMVRefreshInfo; import org.apache.doris.mtmv.MTMVRefreshPartitionSnapshot; import org.apache.doris.mtmv.MTMVRefreshSnapshot; +import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.mtmv.MTMVRelation; +import org.apache.doris.mtmv.MTMVSnapshotIf; import org.apache.doris.mtmv.MTMVStatus; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import java.io.IOException; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -242,6 +246,7 @@ public boolean addTaskResult(MTMVTask task, MTMVRelation relation, this.status.setRefreshState(MTMVRefreshState.FAIL); } this.jobInfo.addHistoryTask(task); + compatiblePctSnapshot(partitionSnapshots); this.refreshSnapshot.updateSnapshots(partitionSnapshots, getPartitionNames()); Env.getCurrentEnv().getMtmvService() .refreshComplete(this, relation, task); @@ -397,64 +402,26 @@ public Map generateMvPartitionDescs() { return result; } - /** - * Calculate the partition and associated partition mapping relationship of the MTMV - * It is the result of real-time comparison calculation, so there may be some costs, - * so it should be called with caution. - * The reason for not directly calling `calculatePartitionMappings` and - * generating a reverse index is to directly generate two maps here, - * without the need to traverse them again - * - * @return mvPartitionName ==> relationPartitionNames and relationPartitionName ==> mvPartitionName - * @throws AnalysisException - */ - public Pair>, Map> calculateDoublyPartitionMappings() - throws AnalysisException { - if (mvPartitionInfo.getPartitionType() == MTMVPartitionType.SELF_MANAGE) { - return Pair.of(Maps.newHashMap(), Maps.newHashMap()); - } - long start = System.currentTimeMillis(); - Map> mvToBase = Maps.newHashMap(); - Map baseToMv = Maps.newHashMap(); - Map> relatedPartitionDescs = MTMVPartitionUtil - .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); - Map mvPartitionItems = getAndCopyPartitionItems(); - for (Entry entry : mvPartitionItems.entrySet()) { - Set basePartitionNames = relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), - Sets.newHashSet()); - String mvPartitionName = entry.getKey(); - mvToBase.put(mvPartitionName, basePartitionNames); - for (String basePartitionName : basePartitionNames) { - baseToMv.put(basePartitionName, mvPartitionName); - } - } - if (LOG.isDebugEnabled()) { - LOG.debug("calculateDoublyPartitionMappings use [{}] mills, mvName is [{}]", - System.currentTimeMillis() - start, name); - } - return Pair.of(mvToBase, baseToMv); - } - /** * Calculate the partition and associated partition mapping relationship of the MTMV * It is the result of real-time comparison calculation, so there may be some costs, * so it should be called with caution * - * @return mvPartitionName ==> relationPartitionNames + * @return mvPartitionName ==> pctTable ==> pctPartitionName * @throws AnalysisException */ - public Map> calculatePartitionMappings() throws AnalysisException { + public Map>> calculatePartitionMappings() throws AnalysisException { if (mvPartitionInfo.getPartitionType() == MTMVPartitionType.SELF_MANAGE) { return Maps.newHashMap(); } long start = System.currentTimeMillis(); - Map> res = Maps.newHashMap(); - Map> relatedPartitionDescs = MTMVPartitionUtil - .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties); + Map>> res = Maps.newHashMap(); + Map>> pctPartitionDescs = MTMVPartitionUtil + .generateRelatedPartitionDescs(mvPartitionInfo, mvProperties, getPartitionColumns()); Map mvPartitionItems = getAndCopyPartitionItems(); for (Entry entry : mvPartitionItems.entrySet()) { res.put(entry.getKey(), - relatedPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Sets.newHashSet())); + pctPartitionDescs.getOrDefault(entry.getValue().toPartitionKeyDesc(), Maps.newHashMap())); } if (LOG.isDebugEnabled()) { LOG.debug("calculatePartitionMappings use [{}] mills, mvName is [{}]", @@ -580,4 +547,28 @@ private void compatibleInternal(CatalogMgr catalogMgr) throws Exception { refreshSnapshot.compatible(this); } } + + @Override + public void gsonPostProcess() throws IOException { + super.gsonPostProcess(); + Map partitionSnapshots = refreshSnapshot.getPartitionSnapshots(); + compatiblePctSnapshot(partitionSnapshots); + } + + private void compatiblePctSnapshot(Map partitionSnapshots) { + BaseTableInfo relatedTableInfo = mvPartitionInfo.getRelatedTableInfo(); + if (relatedTableInfo == null) { + return; + } + if (MapUtils.isEmpty(partitionSnapshots)) { + return; + } + for (MTMVRefreshPartitionSnapshot partitionSnapshot : partitionSnapshots.values()) { + Map partitions = partitionSnapshot.getPartitions(); + Map> pcts = partitionSnapshot.getPcts(); + if (!MapUtils.isEmpty(partitions) && MapUtils.isEmpty(pcts)) { + pcts.put(relatedTableInfo, partitions); + } + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index a0421b3e4450c2..93d64172025ba6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -3418,7 +3418,7 @@ public List getPartitionColumns() { public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, Optional snapshot) throws AnalysisException { - Map partitionVersions = context.getBaseVersions().getPartitionVersions(); + Map partitionVersions = context.getBaseVersions().getPartitionVersions(this); long partitionId = getPartitionOrAnalysisException(partitionName).getId(); long visibleVersion = partitionVersions.containsKey(partitionName) ? partitionVersions.get(partitionName) : getPartitionOrAnalysisException(partitionName).getVisibleVersion(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java index 84367b316cd790..c8b52ab1f5807f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java +++ b/fe/fe-core/src/main/java/org/apache/doris/job/extensions/mtmv/MTMVTask.java @@ -44,6 +44,7 @@ import org.apache.doris.job.exception.JobException; import org.apache.doris.job.task.AbstractTask; import org.apache.doris.metric.MetricRepo; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVBaseTableIf; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; @@ -209,12 +210,14 @@ public void run() throws JobException { MTMVPlanUtil.ensureMTMVQueryUsable(mtmv, ctx); } if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - if (!relatedTable.isValidRelatedTable()) { - throw new JobException("MTMV " + mtmv.getName() + "'s related table " + relatedTable.getName() - + " is not a valid related table anymore, stop refreshing." - + " e.g. Table has multiple partition columns" - + " or including not supported transform functions."); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + for (MTMVRelatedTableIf pctTable : pctTables) { + if (!pctTable.isValidRelatedTable()) { + throw new JobException("MTMV " + mtmv.getName() + "'s pct table " + pctTable.getName() + + " is not a valid pct table anymore, stop refreshing." + + " e.g. Table has multiple partition columns" + + " or including not supported transform functions."); + } } syncPartitions = MTMVPartitionUtil.alignMvPartition(mtmv); } @@ -578,8 +581,11 @@ protected void closeOrReleaseResources() { private Map getIncrementalTableMap() throws AnalysisException { Map tableWithPartKey = Maps.newHashMap(); if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - tableWithPartKey - .put(mtmv.getMvPartitionInfo().getRelatedTable(), mtmv.getMvPartitionInfo().getRelatedCol()); + List pctInfos = mtmv.getMvPartitionInfo().getPctInfos(); + for (BaseColInfo pctInfo : pctInfos) { + tableWithPartKey + .put(MTMVUtil.getTable(pctInfo.getTableInfo()), pctInfo.getColName()); + } } return tableWithPartKey; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseColInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseColInfo.java new file mode 100644 index 00000000000000..16a77b4e4d182f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/BaseColInfo.java @@ -0,0 +1,73 @@ +// 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.mtmv; + +import com.google.gson.annotations.SerializedName; + +import java.util.Objects; + +public class BaseColInfo { + @SerializedName("ti") + private BaseTableInfo tableInfo; + @SerializedName("rn") + private String colName; + + public BaseColInfo(String colName, BaseTableInfo tableInfo) { + this.colName = colName; + this.tableInfo = tableInfo; + } + + public String getColName() { + return colName; + } + + public void setColName(String colName) { + this.colName = colName; + } + + public BaseTableInfo getTableInfo() { + return tableInfo; + } + + public void setTableInfo(BaseTableInfo tableInfo) { + this.tableInfo = tableInfo; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + BaseColInfo that = (BaseColInfo) o; + return Objects.equals(tableInfo, that.tableInfo) && Objects.equals(colName, that.colName); + } + + @Override + public int hashCode() { + return Objects.hash(tableInfo, colName); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("BaseColInfo{"); + sb.append("colName='").append(colName).append('\''); + sb.append(", tableInfo=").append(tableInfo); + sb.append('}'); + return sb.toString(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVBaseVersions.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVBaseVersions.java index 7f83389a953ccf..0e3ce81d38dca6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVBaseVersions.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVBaseVersions.java @@ -17,13 +17,16 @@ package org.apache.doris.mtmv; +import com.google.common.collect.Maps; + import java.util.Map; public class MTMVBaseVersions { private final Map tableVersions; - private final Map partitionVersions; + private final Map> partitionVersions; - public MTMVBaseVersions(Map tableVersions, Map partitionVersions) { + public MTMVBaseVersions(Map tableVersions, + Map> partitionVersions) { this.tableVersions = tableVersions; this.partitionVersions = partitionVersions; } @@ -32,7 +35,7 @@ public Map getTableVersions() { return tableVersions; } - public Map getPartitionVersions() { - return partitionVersions; + public Map getPartitionVersions(MTMVRelatedTableIf mtmvRelatedTableIf) { + return partitionVersions.getOrDefault(mtmvRelatedTableIf, Maps.newHashMap()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionCheckUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionCheckUtil.java index 1f65880583c327..769731155ed839 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionCheckUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionCheckUtil.java @@ -120,7 +120,7 @@ public static Pair compareDynamicPartition(OlapTable originalTa @VisibleForTesting public static Pair compareAutoPartition(OlapTable originalTable, OlapTable relatedTable) throws AnalysisException { - if (!isDynamicPartition(relatedTable)) { + if (!isAutoPartition(relatedTable)) { return Pair.of(false, "relatedTable is not dynamic partition."); } FunctionIntervalInfo originalFunctionIntervalInfo = PartitionExprUtil.getFunctionIntervalInfo( diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java index 38e82298fbd09b..3479e00ca0a990 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprDateTrunc.java @@ -69,18 +69,21 @@ public void analyze(MTMVPartitionInfo mvPartitionInfo) throws AnalysisException throw new AnalysisException( String.format("timeUnit not support: %s, only support: %s", this.timeUnit, timeUnits)); } - MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); - if (partitionType == PartitionType.RANGE) { - Type partitionColumnType = MTMVPartitionUtil - .getPartitionColumnType(mvPartitionInfo.getRelatedTable(), mvPartitionInfo.getRelatedCol()); - if (!partitionColumnType.isDateType()) { - throw new AnalysisException( - "partitionColumnType should be date/datetime " - + "when PartitionType is range and expr is date_trunc"); + List pctInfos = mvPartitionInfo.getPctInfos(); + for (BaseColInfo pctInfo : pctInfos) { + MTMVRelatedTableIf pctTable = MTMVUtil.getRelatedTable(pctInfo.getTableInfo()); + PartitionType partitionType = pctTable.getPartitionType(MvccUtil.getSnapshotFromContext(pctTable)); + if (partitionType == PartitionType.RANGE) { + Type partitionColumnType = MTMVPartitionUtil + .getPartitionColumnType(pctTable, pctInfo.getColName()); + if (!partitionColumnType.isDateType()) { + throw new AnalysisException( + "partitionColumnType should be date/datetime " + + "when PartitionType is range and expr is date_trunc"); + } + } else { + throw new AnalysisException("date_trunc only support range partition"); } - } else { - throw new AnalysisException("date_trunc only support range partition"); } } @@ -125,9 +128,9 @@ private Optional getDateFormat(Map mvProperties) { @Override public PartitionKeyDesc generateRollUpPartitionKeyDesc(PartitionKeyDesc partitionKeyDesc, - MTMVPartitionInfo mvPartitionInfo) throws AnalysisException { + MTMVPartitionInfo mvPartitionInfo, MTMVRelatedTableIf pctTable) throws AnalysisException { Type partitionColumnType = MTMVPartitionUtil - .getPartitionColumnType(mvPartitionInfo.getRelatedTable(), mvPartitionInfo.getRelatedCol()); + .getPartitionColumnType(pctTable, mvPartitionInfo.getPartitionColByPctTable(pctTable)); // mtmv only support one partition column Preconditions.checkState(partitionKeyDesc.getLowerValues().size() == 1, "only support one partition column"); @@ -243,4 +246,18 @@ private String dateTimeToStr(DateTimeV2Literal literal, "MTMV not support partition with column type : " + partitionColumnType); } } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + MTMVPartitionExprDateTrunc that = (MTMVPartitionExprDateTrunc) o; + return Objects.equals(timeUnit, that.timeUnit); + } + + @Override + public int hashCode() { + return Objects.hashCode(timeUnit); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprService.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprService.java index 670368f8954496..901b07fe3dc821 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionExprService.java @@ -47,7 +47,7 @@ String getRollUpIdentity(PartitionKeyDesc partitionKeyDesc, Map * @throws AnalysisException */ PartitionKeyDesc generateRollUpPartitionKeyDesc( - PartitionKeyDesc partitionKeyDesc, MTMVPartitionInfo mvPartitionInfo) + PartitionKeyDesc partitionKeyDesc, MTMVPartitionInfo mvPartitionInfo, MTMVRelatedTableIf pctTable) throws AnalysisException; /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java index 9247b716497967..868c8713ef597f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionInfo.java @@ -22,16 +22,22 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.CatalogMgr; import org.apache.doris.datasource.mvcc.MvccUtil; +import org.apache.doris.persist.gson.GsonPostProcessable; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.gson.annotations.SerializedName; +import org.apache.commons.collections.CollectionUtils; +import java.io.IOException; import java.util.List; import java.util.Objects; +import java.util.Set; /** * MTMVPartitionInfo */ -public class MTMVPartitionInfo { +public class MTMVPartitionInfo implements GsonPostProcessable { public enum MTMVPartitionType { FOLLOW_BASE_TABLE, @@ -41,16 +47,22 @@ public enum MTMVPartitionType { @SerializedName("pt") private MTMVPartitionType partitionType; + // old version only support one pct table + @Deprecated @SerializedName("rt") private BaseTableInfo relatedTable; + @Deprecated @SerializedName("rc") private String relatedCol; @SerializedName("pc") private String partitionCol; @SerializedName("expr") private Expr expr; + @SerializedName("pi") + private List pctInfos = Lists.newArrayList(); public MTMVPartitionInfo() { + this.pctInfos = Lists.newArrayList(); } public MTMVPartitionInfo(MTMVPartitionType partitionType) { @@ -71,14 +83,28 @@ public void setPartitionType(MTMVPartitionType partitionType) { this.partitionType = partitionType; } + @Deprecated public BaseTableInfo getRelatedTableInfo() { return relatedTable; } + @Deprecated public MTMVRelatedTableIf getRelatedTable() throws AnalysisException { return (MTMVRelatedTableIf) MTMVUtil.getTable(relatedTable); } + public Set getPctTables() throws AnalysisException { + Set res = Sets.newHashSetWithExpectedSize(pctInfos.size()); + for (BaseColInfo baseColInfo : pctInfos) { + res.add((MTMVRelatedTableIf) MTMVUtil.getTable(baseColInfo.getTableInfo())); + } + return res; + } + + public List getPctInfos() { + return pctInfos; + } + public void setRelatedTable(BaseTableInfo relatedTable) { this.relatedTable = relatedTable; } @@ -108,50 +134,67 @@ public void setExpr(Expr expr) { } /** - * Get the position of relatedCol in the relatedTable partition column + * Get the position of pct col in the pctTable partition column * * @return * @throws AnalysisException */ - public int getRelatedColPos() throws AnalysisException { + public int getPctColPos(MTMVRelatedTableIf pctTable) throws AnalysisException { if (partitionType == MTMVPartitionType.SELF_MANAGE) { throw new AnalysisException("partitionType is: " + partitionType); } - MTMVRelatedTableIf mtmvRelatedTableIf = getRelatedTable(); - List partitionColumns = mtmvRelatedTableIf.getPartitionColumns( - MvccUtil.getSnapshotFromContext(mtmvRelatedTableIf)); + BaseColInfo pctInfo = getPctInfoByPctTable(pctTable); + List partitionColumns = pctTable.getPartitionColumns( + MvccUtil.getSnapshotFromContext(pctTable)); for (int i = 0; i < partitionColumns.size(); i++) { - if (partitionColumns.get(i).getName().equalsIgnoreCase(relatedCol)) { + if (partitionColumns.get(i).getName().equalsIgnoreCase(pctInfo.getColName())) { return i; } } throw new AnalysisException( - String.format("getRelatedColPos error, relatedCol: %s, partitionColumns: %s", relatedCol, + String.format("getPctColPos error, pctCol: %s, partitionColumns: %s", pctInfo.getColName(), partitionColumns)); } + public String getPartitionColByPctTable(MTMVRelatedTableIf pctTable) throws AnalysisException { + BaseColInfo pctInfoByPctTable = getPctInfoByPctTable(pctTable); + return pctInfoByPctTable.getColName(); + } + + private BaseColInfo getPctInfoByPctTable(MTMVRelatedTableIf pctTable) throws AnalysisException { + BaseTableInfo pctInfo = new BaseTableInfo(pctTable); + for (BaseColInfo baseColInfo : pctInfos) { + if (baseColInfo.getTableInfo().equals(pctInfo)) { + return baseColInfo; + } + } + throw new AnalysisException("not have this pct table"); + } + + public void setPctInfos(List pctInfos) { + this.pctInfos = pctInfos; + } + @Override public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } MTMVPartitionInfo that = (MTMVPartitionInfo) o; - return partitionType == that.partitionType && Objects.equals(relatedTable, that.relatedTable) - && Objects.equals(relatedCol, that.relatedCol) && Objects.equals(partitionCol, - that.partitionCol) && Objects.equals(expr, that.expr); + return partitionType == that.partitionType && Objects.equals(partitionCol, that.partitionCol) + && Objects.equals(expr, that.expr) && Objects.equals(pctInfos, that.pctInfos); } @Override public int hashCode() { - return Objects.hash(partitionType, relatedTable, relatedCol, partitionCol, expr); + return Objects.hash(partitionType, partitionCol, expr, pctInfos); } // toString() is not easy to find where to call the method public String toInfoString() { return "MTMVPartitionInfo{" + "partitionType=" + partitionType - + ", relatedTable=" + relatedTable - + ", relatedCol='" + relatedCol + '\'' + + ", pctInfos=" + pctInfos + ", partitionCol='" + partitionCol + '\'' + ", expr='" + expr + '\'' + '}'; @@ -165,8 +208,7 @@ public String toNameString() { } else { return "MTMVPartitionInfo{" + "partitionType=" + partitionType - + ", relatedTable=" + relatedTable.getTableName() - + ", relatedCol='" + relatedCol + '\'' + + ", pctInfos=" + pctInfos + ", partitionCol='" + partitionCol + '\'' + ", expr='" + expr + '\'' + '}'; @@ -179,4 +221,11 @@ public void compatible(CatalogMgr catalogMgr) throws Exception { } relatedTable.compatible(catalogMgr); } + + @Override + public void gsonPostProcess() throws IOException { + if (relatedTable != null && CollectionUtils.isEmpty(pctInfos)) { + pctInfos.add(new BaseColInfo(relatedCol, relatedTable)); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java index 33ad1defefa5a1..69113034746262 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPartitionUtil.java @@ -44,6 +44,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; +import org.apache.commons.collections.MapUtils; import org.apache.commons.lang3.StringUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -73,7 +74,8 @@ public class MTMVPartitionUtil { new MTMVRelatedPartitionDescInitGenerator(), new MTMVRelatedPartitionDescSyncLimitGenerator(), new MTMVRelatedPartitionDescOnePartitionColGenerator(), - new MTMVRelatedPartitionDescRollUpGenerator() + new MTMVRelatedPartitionDescRollUpGenerator(), + new MTMVRelatedPartitionDescTransferGenerator() ); /** @@ -90,20 +92,25 @@ public static boolean isMTMVPartitionSync(MTMVRefreshContext refreshContext, Str Set tables, Set excludedTriggerTables) throws AnalysisException { MTMV mtmv = refreshContext.getMtmv(); - Set relatedPartitionNames = refreshContext.getPartitionMappings().get(partitionName); - boolean isSyncWithPartition = true; + Map> partitionMappings = refreshContext.getByPartitionName(partitionName); if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - // if follow base table, not need compare with related table, only should compare with related partition - excludedTriggerTables.add(new TableNameInfo(relatedTable)); - if (CollectionUtils.isEmpty(relatedPartitionNames)) { - LOG.warn("can not found related partition, partitionId: {}, mtmvName: {}, relatedTableName: {}", - partitionName, mtmv.getName(), relatedTable.getName()); + if (MapUtils.isEmpty(partitionMappings)) { + LOG.warn("can not found pct partition, partitionName: {}, mtmvName: {}", + partitionName, mtmv.getName()); return false; } - isSyncWithPartition = isSyncWithPartitions(refreshContext, partitionName, relatedPartitionNames); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + for (MTMVRelatedTableIf pctTable : pctTables) { + Set relatedPartitionNames = partitionMappings.getOrDefault(pctTable, Sets.newHashSet()); + // if follow base table, not need compare with related table, only should compare with related partition + excludedTriggerTables.add(new TableNameInfo(pctTable)); + if (!isSyncWithPartitions(refreshContext, partitionName, relatedPartitionNames, pctTable)) { + return false; + } + } + } - return isSyncWithPartition && isSyncWithAllBaseTables(refreshContext, partitionName, tables, + return isSyncWithAllBaseTables(refreshContext, partitionName, tables, excludedTriggerTables); } @@ -118,7 +125,7 @@ public static boolean isMTMVPartitionSync(MTMVRefreshContext refreshContext, Str public static Pair, List> alignMvPartition(MTMV mtmv) throws AnalysisException { Map mtmvPartitionDescs = mtmv.generateMvPartitionDescs(); Set relatedPartitionDescs = generateRelatedPartitionDescs(mtmv.getMvPartitionInfo(), - mtmv.getMvProperties()).keySet(); + mtmv.getMvProperties(), mtmv.getPartitionColumns()).keySet(); List partitionsToDrop = new ArrayList<>(); List partitionsToAdd = new ArrayList<>(); // drop partition of mtmv @@ -142,15 +149,18 @@ public static Pair, List> alignMvPartition(MTMV m * * @param tableProperties * @param mvPartitionInfo + * @param partitionColumns * @return * @throws AnalysisException */ public static List getPartitionDescsByRelatedTable( - Map tableProperties, MTMVPartitionInfo mvPartitionInfo, Map mvProperties) + Map tableProperties, MTMVPartitionInfo mvPartitionInfo, Map mvProperties, + List partitionColumns) throws AnalysisException { List res = Lists.newArrayList(); HashMap partitionProperties = Maps.newHashMap(); - Set relatedPartitionDescs = generateRelatedPartitionDescs(mvPartitionInfo, mvProperties) + Set relatedPartitionDescs = generateRelatedPartitionDescs(mvPartitionInfo, mvProperties, + partitionColumns) .keySet(); for (PartitionKeyDesc partitionKeyDesc : relatedPartitionDescs) { SinglePartitionDesc singlePartitionDesc = new SinglePartitionDesc(true, @@ -163,27 +173,19 @@ public static List getPartitionDescsByRelatedTable( return res; } - public static Map> generateRelatedPartitionDescs(MTMVPartitionInfo mvPartitionInfo, - Map mvProperties) throws AnalysisException { + public static Map>> generateRelatedPartitionDescs( + MTMVPartitionInfo mvPartitionInfo, + Map mvProperties, List partitionColumns) throws AnalysisException { long start = System.currentTimeMillis(); RelatedPartitionDescResult result = new RelatedPartitionDescResult(); for (MTMVRelatedPartitionDescGeneratorService service : partitionDescGenerators) { - service.apply(mvPartitionInfo, mvProperties, result); + service.apply(mvPartitionInfo, mvProperties, result, partitionColumns); } if (LOG.isDebugEnabled()) { LOG.debug("generateRelatedPartitionDescs use [{}] mills, mvPartitionInfo is [{}]", System.currentTimeMillis() - start, mvPartitionInfo); } - return result.getDescs(); - } - - public static List getPartitionsIdsByNames(MTMV mtmv, List partitions) throws AnalysisException { - List res = Lists.newArrayList(); - for (String partitionName : partitions) { - Partition partition = mtmv.getPartitionOrAnalysisException(partitionName); - res.add(partition.getId()); - } - return res; + return result.getRes(); } /** @@ -251,28 +253,25 @@ public static Map> getPartitionsUnSyncTables(MTMV mtmv) private static List getPartitionUnSyncTables(MTMVRefreshContext context, String partitionName) throws AnalysisException { MTMV mtmv = context.getMtmv(); - Set relatedPartitionNames = context.getPartitionMappings().get(partitionName); + Map> mappings = context.getByPartitionName(partitionName); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); List res = Lists.newArrayList(); for (BaseTableInfo baseTableInfo : mtmv.getRelation().getBaseTablesOneLevelAndFromView()) { TableIf table = MTMVUtil.getTable(baseTableInfo); if (!(table instanceof MTMVRelatedTableIf)) { continue; } - MTMVRelatedTableIf mtmvRelatedTableIf = (MTMVRelatedTableIf) table; - if (!mtmvRelatedTableIf.needAutoRefresh()) { + MTMVRelatedTableIf pctTable = (MTMVRelatedTableIf) table; + if (!pctTable.needAutoRefresh()) { continue; } - if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE && mtmv - .getMvPartitionInfo().getRelatedTableInfo().equals(baseTableInfo)) { - if (CollectionUtils.isEmpty(relatedPartitionNames)) { - // can not found related partition - res.add(mtmvRelatedTableIf.getName()); - continue; - } + if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE && pctTables.contains( + pctTable)) { + Set pctPartitions = mappings.getOrDefault(pctTable, Sets.newHashSet()); boolean isSyncWithPartition = isSyncWithPartitions(context, partitionName, - relatedPartitionNames); + pctPartitions, pctTable); if (!isSyncWithPartition) { - res.add(mtmvRelatedTableIf.getName()); + res.add(pctTable.getName()); } } else { if (!isSyncWithBaseTable(context, partitionName, baseTableInfo)) { @@ -313,34 +312,39 @@ public static List getMTMVNeedRefreshPartitions(MTMVRefreshContext conte * * @param context * @param mtmvPartitionName - * @param relatedPartitionNames + * @param pctPartitionNames * @return * @throws AnalysisException */ public static boolean isSyncWithPartitions(MTMVRefreshContext context, String mtmvPartitionName, - Set relatedPartitionNames) throws AnalysisException { + Set pctPartitionNames, MTMVRelatedTableIf pctTable) throws AnalysisException { MTMV mtmv = context.getMtmv(); - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - if (!relatedTable.needAutoRefresh()) { + if (!pctTable.needAutoRefresh()) { return true; } // check if partitions of related table is changed - Set snapshotPartitions = mtmv.getRefreshSnapshot().getSnapshotPartitions(mtmvPartitionName); - if (!Objects.equals(relatedPartitionNames, snapshotPartitions)) { + BaseTableInfo pctTableInfo = new BaseTableInfo(pctTable); + // check if partitions of related table is changed + Set snapshotPartitions = mtmv.getRefreshSnapshot() + .getPctSnapshots(mtmvPartitionName, pctTableInfo); + if (!Objects.equals(pctPartitionNames, snapshotPartitions)) { return false; } - for (String relatedPartitionName : relatedPartitionNames) { - MTMVSnapshotIf relatedPartitionCurrentSnapshot = relatedTable - .getPartitionSnapshot(relatedPartitionName, context, MvccUtil.getSnapshotFromContext(relatedTable)); + if (CollectionUtils.isEmpty(pctPartitionNames)) { + return true; + } + for (String pctPartitionName : pctPartitionNames) { + MTMVSnapshotIf pctCurrentSnapshot = pctTable + .getPartitionSnapshot(pctPartitionName, context, MvccUtil.getSnapshotFromContext(pctTable)); if (LOG.isDebugEnabled()) { LOG.debug(String.format("isSyncWithPartitions mvName is %s\n, mtmvPartitionName is %s\n, " - + "mtmv refreshSnapshot is %s\n, relatedPartitionName is %s\n, " - + "relatedPartitionCurrentSnapshot is %s", mtmv.getName(), mtmvPartitionName, - mtmv.getRefreshSnapshot(), relatedPartitionName, relatedPartitionCurrentSnapshot)); + + "mtmv refreshSnapshot is %s\n, pctPartitionName is %s\n, " + + "pctCurrentSnapshot is %s", mtmv.getName(), mtmvPartitionName, + mtmv.getRefreshSnapshot(), pctPartitionName, pctCurrentSnapshot)); } if (!mtmv.getRefreshSnapshot() - .equalsWithRelatedPartition(mtmvPartitionName, relatedPartitionName, - relatedPartitionCurrentSnapshot)) { + .equalsWithPct(mtmvPartitionName, pctPartitionName, + pctCurrentSnapshot, pctTableInfo)) { return false; } } @@ -543,24 +547,37 @@ public static Map generatePartitionSnapsho private static MTMVRefreshPartitionSnapshot generatePartitionSnapshot(MTMVRefreshContext context, - Set baseTables, Set relatedPartitionNames) + Set baseTables, Map> pctPartitionNames) throws AnalysisException { MTMV mtmv = context.getMtmv(); MTMVRefreshPartitionSnapshot refreshPartitionSnapshot = new MTMVRefreshPartitionSnapshot(); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE) { - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - for (String relatedPartitionName : relatedPartitionNames) { - MTMVSnapshotIf partitionSnapshot = relatedTable.getPartitionSnapshot(relatedPartitionName, context, - MvccUtil.getSnapshotFromContext(relatedTable)); - refreshPartitionSnapshot.getPartitions().put(relatedPartitionName, partitionSnapshot); + for (MTMVRelatedTableIf pctTable : pctTables) { + Map pctSnapshot = refreshPartitionSnapshot.getPctSnapshot( + new BaseTableInfo(pctTable)); + Set oneTablePartitionNames = pctPartitionNames.get(pctTable); + if (CollectionUtils.isEmpty(oneTablePartitionNames)) { + continue; + } + for (String pctPartitionName : oneTablePartitionNames) { + MTMVSnapshotIf partitionSnapshot = pctTable.getPartitionSnapshot(pctPartitionName, context, + MvccUtil.getSnapshotFromContext(pctTable)); + pctSnapshot.put(pctPartitionName, partitionSnapshot); + } + } + // compatible old version + if (pctTables.size() == 1) { + refreshPartitionSnapshot.getPartitions() + .putAll(refreshPartitionSnapshot.getPcts().entrySet().iterator().next().getValue()); } } for (BaseTableInfo baseTableInfo : baseTables) { - if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE && mtmv - .getMvPartitionInfo().getRelatedTableInfo().equals(baseTableInfo)) { + TableIf table = MTMVUtil.getTable(baseTableInfo); + if (mtmv.getMvPartitionInfo().getPartitionType() != MTMVPartitionType.SELF_MANAGE && pctTables.contains( + table)) { continue; } - TableIf table = MTMVUtil.getTable(baseTableInfo); if (!(table instanceof MTMVRelatedTableIf)) { continue; } @@ -584,25 +601,29 @@ public static MTMVBaseVersions getBaseVersions(MTMV mtmv) throws AnalysisExcepti return new MTMVBaseVersions(getTableVersions(mtmv), getPartitionVersions(mtmv)); } - private static Map getPartitionVersions(MTMV mtmv) throws AnalysisException { - Map res = Maps.newHashMap(); + private static Map> getPartitionVersions(MTMV mtmv) throws AnalysisException { + Map> res = Maps.newHashMap(); if (mtmv.getMvPartitionInfo().getPartitionType().equals(MTMVPartitionType.SELF_MANAGE)) { return res; } - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - if (!(relatedTable instanceof OlapTable)) { - return res; - } - List partitions = Lists.newArrayList(((OlapTable) relatedTable).getPartitions()); - List versions = null; - try { - versions = Partition.getVisibleVersions(partitions); - } catch (RpcException e) { - throw new AnalysisException("getVisibleVersions failed.", e); - } - Preconditions.checkState(partitions.size() == versions.size()); - for (int i = 0; i < partitions.size(); i++) { - res.put(partitions.get(i).getName(), versions.get(i)); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + for (MTMVRelatedTableIf pctTable : pctTables) { + if (!(pctTable instanceof OlapTable)) { + continue; + } + Map onePctResult = Maps.newHashMap(); + List partitions = Lists.newArrayList(((OlapTable) pctTable).getPartitions()); + List versions = null; + try { + versions = Partition.getVisibleVersions(partitions); + } catch (RpcException e) { + throw new AnalysisException("getVisibleVersions failed.", e); + } + Preconditions.checkState(partitions.size() == versions.size()); + for (int i = 0; i < partitions.size(); i++) { + onePctResult.put(partitions.get(i).getName(), versions.get(i)); + } + res.put(pctTable, onePctResult); } return res; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java index 77602b49337e7a..a29915a846166c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVPlanUtil.java @@ -42,6 +42,7 @@ import org.apache.doris.common.util.Util; import org.apache.doris.datasource.CatalogIf; import org.apache.doris.job.exception.JobException; +import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.analyzer.UnboundResultSink; @@ -81,6 +82,8 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.util.Arrays; import java.util.List; @@ -91,6 +94,7 @@ import javax.annotation.Nullable; public class MTMVPlanUtil { + private static final Logger LOG = LogManager.getLogger(MTMVPlanUtil.class); public static ConnectContext createMTMVContext(MTMV mtmv) { ConnectContext ctx = createBasicMvContext(null); @@ -540,12 +544,39 @@ public static void ensureMTMVQueryUsable(MTMV mtmv, ConnectContext ctx) throws J private static void checkMTMVPartitionInfo(MTMV mtmv, MTMVPartitionInfo analyzedMvPartitionInfo) throws JobException { MTMVPartitionInfo originalMvPartitionInfo = mtmv.getMvPartitionInfo(); - if (!analyzedMvPartitionInfo.equals(originalMvPartitionInfo)) { + if (!checkMTMVPartitionInfoLike(originalMvPartitionInfo, analyzedMvPartitionInfo)) { throw new JobException("async materialized view partition info changed, analyzed: %s, original: %s", analyzedMvPartitionInfo.toInfoString(), originalMvPartitionInfo.toInfoString()); } } + private static boolean checkMTMVPartitionInfoLike(MTMVPartitionInfo originalMvPartitionInfo, + MTMVPartitionInfo analyzedMvPartitionInfo) { + if (!originalMvPartitionInfo.getPartitionType().equals(analyzedMvPartitionInfo.getPartitionType())) { + return false; + } + if (originalMvPartitionInfo.getPartitionType() == MTMVPartitionType.SELF_MANAGE) { + return true; + } + // because old version only support one pct table, so can not use equal + if (!analyzedMvPartitionInfo.getPctInfos().containsAll(originalMvPartitionInfo.getPctInfos())) { + return false; + } + if (originalMvPartitionInfo.getPartitionType() == MTMVPartitionType.EXPR) { + try { + MTMVPartitionExprService originalExprService = MTMVPartitionExprFactory.getExprService( + originalMvPartitionInfo.getExpr()); + MTMVPartitionExprService analyzedExprService = MTMVPartitionExprFactory.getExprService( + analyzedMvPartitionInfo.getExpr()); + return originalExprService.equals(analyzedExprService); + } catch (org.apache.doris.common.AnalysisException e) { + LOG.warn(e); + return false; + } + } + return true; + } + private static void checkColumnIfChange(MTMV mtmv, List analyzedColumnDefinitions) throws JobException { List analyzedColumns = analyzedColumnDefinitions.stream() diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshContext.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshContext.java index c59abd9ebdcda7..603f89bbec5d85 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshContext.java @@ -27,7 +27,7 @@ public class MTMVRefreshContext { private MTMV mtmv; - private Map> partitionMappings; + private Map>> partitionMappings; private MTMVBaseVersions baseVersions; // Within the same context, repeated fetches of the same table's snapshot must return consistent values. // Hence, the results are cached at this stage. @@ -42,10 +42,14 @@ public MTMV getMtmv() { return mtmv; } - public Map> getPartitionMappings() { + public Map>> getPartitionMappings() { return partitionMappings; } + public Map> getByPartitionName(String partitionName) { + return partitionMappings.getOrDefault(partitionName, Maps.newHashMap()); + } + public MTMVBaseVersions getBaseVersions() { return baseVersions; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java index cae7d26c77809f..45e842bf9680de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshPartitionSnapshot.java @@ -37,8 +37,12 @@ public class MTMVRefreshPartitionSnapshot { private static final Logger LOG = LogManager.getLogger(MTMV.class); + // old version only support one pct table + @Deprecated @SerializedName("p") private Map partitions; + @SerializedName("pcts") + private Map> pcts; // old version only persist table id, we need `BaseTableInfo`, `tables` only for compatible old version @SerializedName("t") @Deprecated @@ -48,14 +52,20 @@ public class MTMVRefreshPartitionSnapshot { public MTMVRefreshPartitionSnapshot() { this.partitions = Maps.newConcurrentMap(); + this.pcts = Maps.newConcurrentMap(); this.tables = Maps.newConcurrentMap(); this.tablesInfo = Maps.newConcurrentMap(); } + @Deprecated public Map getPartitions() { return partitions; } + public Map getPctSnapshot(BaseTableInfo pctTable) { + return pcts.computeIfAbsent(pctTable, k -> Maps.newHashMap()); + } + public MTMVSnapshotIf getTableSnapshot(BaseTableInfo table) { if (tablesInfo.containsKey(table)) { return tablesInfo.get(table); @@ -70,6 +80,10 @@ public void addTableSnapshot(BaseTableInfo baseTableInfo, MTMVSnapshotIf tableSn tables.put(baseTableInfo.getTableId(), tableSnapshot); } + public Map> getPcts() { + return pcts; + } + @Override public String toString() { return "MTMVRefreshPartitionSnapshot{" diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSnapshot.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSnapshot.java index 0d9665cb4463b9..a86a6d4b24e3b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSnapshot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRefreshSnapshot.java @@ -36,25 +36,26 @@ public MTMVRefreshSnapshot() { this.partitionSnapshots = Maps.newConcurrentMap(); } - public boolean equalsWithRelatedPartition(String mtmvPartitionName, String relatedPartitionName, - MTMVSnapshotIf relatedPartitionCurrentSnapshot) { + public boolean equalsWithPct(String mtmvPartitionName, String pctPartitionName, + MTMVSnapshotIf pctPartitionCurrentSnapshot, BaseTableInfo pctTableInfo) { MTMVRefreshPartitionSnapshot partitionSnapshot = partitionSnapshots.get(mtmvPartitionName); if (partitionSnapshot == null) { return false; } - MTMVSnapshotIf relatedPartitionSnapshot = partitionSnapshot.getPartitions().get(relatedPartitionName); - if (relatedPartitionSnapshot == null) { + MTMVSnapshotIf pctPartitionSnapshot = partitionSnapshot.getPctSnapshot(pctTableInfo) + .get(pctPartitionName); + if (pctPartitionSnapshot == null) { return false; } - return relatedPartitionSnapshot.equals(relatedPartitionCurrentSnapshot); + return pctPartitionSnapshot.equals(pctPartitionCurrentSnapshot); } - public Set getSnapshotPartitions(String mtmvPartitionName) { + public Set getPctSnapshots(String mtmvPartitionName, BaseTableInfo pctTableInfo) { MTMVRefreshPartitionSnapshot partitionSnapshot = partitionSnapshots.get(mtmvPartitionName); if (partitionSnapshot == null) { return Sets.newHashSet(); } - return partitionSnapshot.getPartitions().keySet(); + return partitionSnapshot.getPctSnapshot(pctTableInfo).keySet(); } public boolean equalsWithBaseTable(String mtmvPartitionName, BaseTableInfo tableInfo, @@ -84,6 +85,10 @@ public void updateSnapshots(Map addPartiti } } + public Map getPartitionSnapshots() { + return partitionSnapshots; + } + @Override public String toString() { return "MTMVRefreshSnapshot{" diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorService.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorService.java index 09d85576b5cba4..5760995ac64e88 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorService.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorService.java @@ -17,8 +17,10 @@ package org.apache.doris.mtmv; +import org.apache.doris.catalog.Column; import org.apache.doris.common.AnalysisException; +import java.util.List; import java.util.Map; /** @@ -31,8 +33,9 @@ public interface MTMVRelatedPartitionDescGeneratorService { * @param mvPartitionInfo PartitionInfo of MTMV * @param mvProperties properties of MTMV * @param lastResult the processing result of the previous process + * @param partitionColumns partitionColumns of mv * @throws AnalysisException */ void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, - RelatedPartitionDescResult lastResult) throws AnalysisException; + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java index 28900f38e59f62..82d45f55288071 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescInitGenerator.java @@ -17,10 +17,16 @@ package org.apache.doris.mtmv; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PartitionItem; import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.mvcc.MvccUtil; +import com.google.common.collect.Maps; + +import java.util.List; import java.util.Map; +import java.util.Set; /** * get all related partition descs @@ -29,8 +35,13 @@ public class MTMVRelatedPartitionDescInitGenerator implements MTMVRelatedPartiti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, - RelatedPartitionDescResult lastResult) throws AnalysisException { - MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - lastResult.setItems(relatedTable.getAndCopyPartitionItems(MvccUtil.getSnapshotFromContext(relatedTable))); + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException { + Set relatedTables = mvPartitionInfo.getPctTables(); + Map> items = Maps.newHashMap(); + for (MTMVRelatedTableIf relatedTable : relatedTables) { + items.put(relatedTable, + relatedTable.getAndCopyPartitionItems(MvccUtil.getSnapshotFromContext(relatedTable))); + } + lastResult.setItems(items); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescOnePartitionColGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescOnePartitionColGenerator.java index c5dad9bdb41891..22148d0090f75c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescOnePartitionColGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescOnePartitionColGenerator.java @@ -18,6 +18,7 @@ package org.apache.doris.mtmv; import org.apache.doris.analysis.PartitionKeyDesc; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.common.AnalysisException; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; @@ -25,6 +26,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; @@ -47,21 +49,26 @@ public class MTMVRelatedPartitionDescOnePartitionColGenerator implements MTMVRel @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, - RelatedPartitionDescResult lastResult) throws AnalysisException { + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException { if (mvPartitionInfo.getPartitionType() == MTMVPartitionType.SELF_MANAGE) { return; } - Map> res = Maps.newHashMap(); - Map relatedPartitionItems = lastResult.getItems(); - int relatedColPos = mvPartitionInfo.getRelatedColPos(); - for (Entry entry : relatedPartitionItems.entrySet()) { - PartitionKeyDesc partitionKeyDesc = entry.getValue().toPartitionKeyDesc(relatedColPos); - if (res.containsKey(partitionKeyDesc)) { - res.get(partitionKeyDesc).add(entry.getKey()); - } else { - res.put(partitionKeyDesc, Sets.newHashSet(entry.getKey())); + Map>> res = Maps.newHashMap(); + Map> relatedPartitionItems = lastResult.getItems(); + for (Entry> entry : relatedPartitionItems.entrySet()) { + int relatedColPos = mvPartitionInfo.getPctColPos(entry.getKey()); + Map> onePctRes = Maps.newHashMap(); + for (Entry onePctEntry : entry.getValue().entrySet()) { + PartitionKeyDesc partitionKeyDesc = onePctEntry.getValue().toPartitionKeyDesc(relatedColPos); + if (onePctRes.containsKey(partitionKeyDesc)) { + onePctRes.get(partitionKeyDesc).add(onePctEntry.getKey()); + } else { + onePctRes.put(partitionKeyDesc, Sets.newHashSet(onePctEntry.getKey())); + } } + res.put(entry.getKey(), onePctRes); } + lastResult.setDescs(res); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java index 71f7fc358f5975..e20910fb571fab 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGenerator.java @@ -19,6 +19,7 @@ import org.apache.doris.analysis.PartitionKeyDesc; import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PartitionType; import org.apache.doris.common.AnalysisException; import org.apache.doris.datasource.mvcc.MvccUtil; @@ -41,16 +42,27 @@ public class MTMVRelatedPartitionDescRollUpGenerator implements MTMVRelatedParti @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, - RelatedPartitionDescResult lastResult) throws AnalysisException { + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException { if (mvPartitionInfo.getPartitionType() != MTMVPartitionType.EXPR) { return; } - MTMVRelatedTableIf relatedTable = mvPartitionInfo.getRelatedTable(); - PartitionType partitionType = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); + Map>> descs = lastResult.getDescs(); + Map>> res = Maps.newHashMap(); + for (Entry>> entry : descs.entrySet()) { + MTMVRelatedTableIf pctTable = entry.getKey(); + res.put(pctTable, rollUpOnePctTable(mvPartitionInfo, mvProperties, pctTable, entry.getValue())); + } + lastResult.setDescs(res); + } + + private Map> rollUpOnePctTable(MTMVPartitionInfo mvPartitionInfo, + Map mvProperties, MTMVRelatedTableIf pctTable, Map> descs) + throws AnalysisException { + PartitionType partitionType = pctTable.getPartitionType(MvccUtil.getSnapshotFromContext(pctTable)); if (partitionType == PartitionType.RANGE) { - lastResult.setDescs(rollUpRange(lastResult.getDescs(), mvPartitionInfo)); + return rollUpRange(descs, mvPartitionInfo, pctTable); } else if (partitionType == PartitionType.LIST) { - lastResult.setDescs(rollUpList(lastResult.getDescs(), mvPartitionInfo, mvProperties)); + return rollUpList(descs, mvPartitionInfo, mvProperties); } else { throw new AnalysisException("only RANGE/LIST partition support roll up"); } @@ -127,16 +139,13 @@ private Set getStringValues(PartitionKeyDesc partitionKeyDesc) { * @throws AnalysisException */ public Map> rollUpRange(Map> relatedPartitionDescs, - MTMVPartitionInfo mvPartitionInfo) throws AnalysisException { + MTMVPartitionInfo mvPartitionInfo, MTMVRelatedTableIf pctTable) throws AnalysisException { Map> result = Maps.newHashMap(); MTMVPartitionExprService exprSerice = MTMVPartitionExprFactory.getExprService(mvPartitionInfo.getExpr()); for (Entry> entry : relatedPartitionDescs.entrySet()) { - PartitionKeyDesc rollUpDesc = exprSerice.generateRollUpPartitionKeyDesc(entry.getKey(), mvPartitionInfo); - if (result.containsKey(rollUpDesc)) { - result.get(rollUpDesc).addAll(entry.getValue()); - } else { - result.put(rollUpDesc, entry.getValue()); - } + PartitionKeyDesc rollUpDesc = exprSerice.generateRollUpPartitionKeyDesc(entry.getKey(), mvPartitionInfo, + pctTable); + result.computeIfAbsent(rollUpDesc, k -> Sets.newHashSet()).addAll(entry.getValue()); } return result; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescSyncLimitGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescSyncLimitGenerator.java index 02f8b087cc9c10..b66cf822516a03 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescSyncLimitGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescSyncLimitGenerator.java @@ -17,6 +17,7 @@ package org.apache.doris.mtmv; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PartitionItem; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.util.PropertyAnalyzer; @@ -32,6 +33,7 @@ import com.google.common.collect.Maps; import org.apache.commons.lang3.StringUtils; +import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Optional; @@ -43,21 +45,26 @@ public class MTMVRelatedPartitionDescSyncLimitGenerator implements MTMVRelatedPa @Override public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, - RelatedPartitionDescResult lastResult) throws AnalysisException { - Map partitionItems = lastResult.getItems(); + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException { + Map> partitionItems = lastResult.getItems(); MTMVPartitionSyncConfig config = generateMTMVPartitionSyncConfigByProperties(mvProperties); if (config.getSyncLimit() <= 0) { return; } long nowTruncSubSec = getNowTruncSubSec(config.getTimeUnit(), config.getSyncLimit()); Optional dateFormat = config.getDateFormat(); - Map res = Maps.newHashMap(); - int relatedColPos = mvPartitionInfo.getRelatedColPos(); - for (Entry entry : partitionItems.entrySet()) { - if (entry.getValue().isGreaterThanSpecifiedTime(relatedColPos, dateFormat, nowTruncSubSec)) { - res.put(entry.getKey(), entry.getValue()); + Map> res = Maps.newHashMap(); + for (Entry> entry : partitionItems.entrySet()) { + Map onePctRes = Maps.newHashMap(); + int relatedColPos = mvPartitionInfo.getPctColPos(entry.getKey()); + for (Entry onePctEntry : entry.getValue().entrySet()) { + if (onePctEntry.getValue().isGreaterThanSpecifiedTime(relatedColPos, dateFormat, nowTruncSubSec)) { + onePctRes.put(onePctEntry.getKey(), onePctEntry.getValue()); + } } + res.put(entry.getKey(), onePctRes); } + lastResult.setItems(res); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescTransferGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescTransferGenerator.java new file mode 100644 index 00000000000000..c301e5e7fd3d2a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescTransferGenerator.java @@ -0,0 +1,132 @@ +// 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.mtmv; + +import org.apache.doris.analysis.PartitionKeyDesc; +import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType; +import org.apache.doris.analysis.PartitionValue; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.PartitionKey; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.util.RangeUtils; + +import com.google.common.collect.Maps; +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.util.Lists; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +/** + * transfer and valid + */ +public class MTMVRelatedPartitionDescTransferGenerator implements MTMVRelatedPartitionDescGeneratorService { + + @Override + public void apply(MTMVPartitionInfo mvPartitionInfo, Map mvProperties, + RelatedPartitionDescResult lastResult, List partitionColumns) throws AnalysisException { + Map>> descs = lastResult.getDescs(); + Map>> res = Maps.newHashMap(); + for (Entry>> entry : descs.entrySet()) { + MTMVRelatedTableIf pctTable = entry.getKey(); + Map> onePctDescs = entry.getValue(); + for (Entry> onePctEntry : onePctDescs.entrySet()) { + PartitionKeyDesc partitionKeyDesc = onePctEntry.getKey(); + Set partitionNames = onePctEntry.getValue(); + Map> partitionKeyDescMap = res.computeIfAbsent(partitionKeyDesc, + k -> new HashMap<>()); + partitionKeyDescMap.put(pctTable, partitionNames); + } + } + if (mvPartitionInfo.getPctInfos().size() > 1) { + checkIntersect(res.keySet(), partitionColumns); + } + lastResult.setRes(res); + } + + public void checkIntersect(Set partitionKeyDescs, List partitionColumns) + throws AnalysisException { + if (CollectionUtils.isEmpty(partitionKeyDescs)) { + return; + } + if (partitionKeyDescs.iterator().next().getPartitionType().equals(PartitionKeyValueType.IN)) { + checkIntersectForList(partitionKeyDescs, partitionColumns); + } else { + checkIntersectForRange(partitionKeyDescs, partitionColumns); + } + } + + public void checkIntersectForList(Set partitionKeyDescs, List partitionColumns) + throws AnalysisException { + Set allPartitionValues = Sets.newHashSet(); + for (PartitionKeyDesc partitionKeyDesc : partitionKeyDescs) { + if (!partitionKeyDesc.hasInValues()) { + throw new AnalysisException("must have in values"); + } + for (List values : partitionKeyDesc.getInValues()) { + for (PartitionValue partitionValue : values) { + if (allPartitionValues.contains(partitionValue)) { + throw new AnalysisException("PartitionValue is repeat: " + partitionValue.getStringValue()); + } else { + allPartitionValues.add(partitionValue); + } + } + } + } + } + + public void checkIntersectForRange(Set partitionKeyDescs, List partitionColumns) + throws AnalysisException { + List> sortedRanges = Lists.newArrayListWithCapacity(partitionKeyDescs.size()); + for (PartitionKeyDesc partitionKeyDesc : partitionKeyDescs) { + if (partitionKeyDesc.hasInValues()) { + throw new AnalysisException("only support range partition"); + } + if (partitionKeyDesc.getPartitionType() != PartitionKeyDesc.PartitionKeyValueType.FIXED) { + throw new AnalysisException("only support fixed partition"); + } + PartitionKey lowKey = PartitionKey.createPartitionKey(partitionKeyDesc.getLowerValues(), partitionColumns); + PartitionKey upperKey = PartitionKey.createPartitionKey(partitionKeyDesc.getUpperValues(), + partitionColumns); + if (lowKey.compareTo(upperKey) >= 0) { + throw new AnalysisException("The lower values must smaller than upper values"); + } + Range range = Range.closedOpen(lowKey, upperKey); + sortedRanges.add(range); + } + + sortedRanges.sort((r1, r2) -> { + return r1.lowerEndpoint().compareTo(r2.lowerEndpoint()); + }); + if (sortedRanges.size() < 2) { + return; + } + for (int i = 0; i < sortedRanges.size() - 1; i++) { + Range current = sortedRanges.get(i); + Range next = sortedRanges.get(i + 1); + if (RangeUtils.checkIsTwoRangesIntersect(current, next)) { + throw new AnalysisException("Range " + current + " is intersected with range: " + next); + } + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java index afa66a48d8ceea..6c8ba97eae6dfb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRelationManager.java @@ -102,9 +102,7 @@ public Set getAvailableMTMVs(Set candidateMTMVs, ConnectContext ctx, if (!mtmv.isUseForRewrite()) { continue; } - BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); - if (isMVPartitionValid(mtmv, ctx, forceConsistent, - relatedTableInfo == null ? null : queryUsedPartitions.get(relatedTableInfo.toList()))) { + if (isMVPartitionValid(mtmv, ctx, forceConsistent, queryUsedPartitions)) { res.add(mtmv); } } @@ -133,10 +131,10 @@ public Set getCandidateMTMVs(List tableInfos) { @VisibleForTesting public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) { long currentTimeMillis = System.currentTimeMillis(); Collection mtmvCanRewritePartitions = MTMVRewriteUtil.getMTMVCanRewritePartitions( - mtmv, ctx, currentTimeMillis, forceConsistent, relatedPartitions); + mtmv, ctx, currentTimeMillis, forceConsistent, queryUsedPartitions); // MTMVRewriteUtil.getMTMVCanRewritePartitions is time-consuming behavior, So record for used later ctx.getStatementContext().getMvCanRewritePartitionsMap().putIfAbsent( new BaseTableInfo(mtmv), mtmvCanRewritePartitions); diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java index 63707c314ea7e4..db14440a02a1b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVRewriteUtil.java @@ -19,9 +19,13 @@ import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; import org.apache.doris.qe.ConnectContext; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; @@ -46,7 +50,7 @@ public class MTMVRewriteUtil { */ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, ConnectContext ctx, long currentTimeMills, boolean forceConsistent, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) { List res = Lists.newArrayList(); Collection allPartitions = mtmv.getPartitions(); MTMVRelation mtmvRelation = mtmv.getRelation(); @@ -57,10 +61,6 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne if (!mtmv.canBeCandidate()) { return res; } - // if relatedPartitions is empty but not null, which means query no partitions - if (relatedPartitions != null && relatedPartitions.size() == 0) { - return res; - } Set mtmvNeedComparePartitions = null; MTMVRefreshContext refreshContext = null; // check gracePeriod @@ -81,8 +81,13 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne } } if (mtmvNeedComparePartitions == null) { - mtmvNeedComparePartitions = getMtmvPartitionsByRelatedPartitions(mtmv, refreshContext, - relatedPartitions); + try { + mtmvNeedComparePartitions = getMtmvPartitionsByRelatedPartitions(mtmv, refreshContext, + queryUsedPartitions); + } catch (AnalysisException e) { + LOG.warn(e); + return res; + } } // if the partition which query not used, should not compare partition version if (!mtmvNeedComparePartitions.contains(partition.getName())) { @@ -103,25 +108,49 @@ public static Collection getMTMVCanRewritePartitions(MTMV mtmv, Conne } private static Set getMtmvPartitionsByRelatedPartitions(MTMV mtmv, MTMVRefreshContext refreshContext, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) throws AnalysisException { + if (mtmv.getMvPartitionInfo().getPartitionType().equals(MTMVPartitionType.SELF_MANAGE)) { + return mtmv.getPartitionNames(); + } // if relatedPartitions is null, which means QueryPartitionCollector visitLogicalCatalogRelation can not // get query used partitions, should get all mtmv partitions - if (relatedPartitions == null) { + if (queryUsedPartitions == null) { return mtmv.getPartitionNames(); } Set res = Sets.newHashSet(); - Map relatedToMv = getRelatedToMv(refreshContext.getPartitionMappings()); - for (String relatedPartition : relatedPartitions) { - res.add(relatedToMv.get(relatedPartition)); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + Map, String> relatedToMv = getPctToMv( + refreshContext.getPartitionMappings()); + for (Entry, Set> entry : queryUsedPartitions.entrySet()) { + TableIf tableIf = MTMVUtil.getTable(entry.getKey()); + if (!pctTables.contains(tableIf)) { + continue; + } + if (entry.getValue() == null) { + return mtmv.getPartitionNames(); + } + Set pctPartitions = entry.getValue(); + for (String pctPartition : pctPartitions) { + String mvPartition = relatedToMv.get(Pair.of(tableIf, pctPartition)); + if (mvPartition != null) { + res.add(mvPartition); + } + } } return res; } - private static Map getRelatedToMv(Map> mvToRelated) { - Map res = Maps.newHashMap(); - for (Entry> entry : mvToRelated.entrySet()) { - for (String relatedPartition : entry.getValue()) { - res.put(relatedPartition, entry.getKey()); + @VisibleForTesting + public static Map, String> getPctToMv( + Map>> partitionMappings) { + Map, String> res = Maps.newHashMap(); + for (Entry>> entry : partitionMappings.entrySet()) { + String mvPartitionName = entry.getKey(); + for (Entry> entry2 : entry.getValue().entrySet()) { + MTMVRelatedTableIf pctTable = entry2.getKey(); + for (String pctPartitionName : entry2.getValue()) { + res.put(Pair.of(pctTable, pctPartitionName), mvPartitionName); + } } } return res; diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java index 01a08f823c51c2..b83442a03b6463 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/MTMVUtil.java @@ -98,6 +98,16 @@ public static MTMV getMTMV(long dbId, long mtmvId) throws DdlException, MetaNotF return (MTMV) db.getTableOrMetaException(mtmvId, TableType.MATERIALIZED_VIEW); } + public static TableIf getTable(List names) throws AnalysisException { + if (names == null || names.size() != 3) { + throw new AnalysisException("size of names need 3, but names is:" + names); + } + return Env.getCurrentEnv().getCatalogMgr() + .getCatalogOrAnalysisException(names.get(0)) + .getDbOrAnalysisException(names.get(1)) + .getTableOrAnalysisException(names.get(2)); + } + /** * if base tables of mtmv contains external table * diff --git a/fe/fe-core/src/main/java/org/apache/doris/mtmv/RelatedPartitionDescResult.java b/fe/fe-core/src/main/java/org/apache/doris/mtmv/RelatedPartitionDescResult.java index b349722a76d6b1..ee5fd532d1b812 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/mtmv/RelatedPartitionDescResult.java +++ b/fe/fe-core/src/main/java/org/apache/doris/mtmv/RelatedPartitionDescResult.java @@ -27,27 +27,40 @@ public class RelatedPartitionDescResult { // PartitionKeyDesc to relatedTable partition ids(Different partitions may have the same PartitionKeyDesc) - private Map> descs; - private Map items; + private Map>> descs; + private Map> items; + private Map>> res; public RelatedPartitionDescResult() { this.descs = Maps.newHashMap(); this.items = Maps.newHashMap(); + this.res = Maps.newHashMap(); } - public Map> getDescs() { + public Map>> getDescs() { return descs; } - public void setDescs(Map> descs) { + public void setDescs( + Map>> descs) { this.descs = descs; } - public Map getItems() { + public Map> getItems() { return items; } - public void setItems(Map items) { + public void setItems( + Map> items) { this.items = items; } + + public Map>> getRes() { + return res; + } + + public void setRes( + Map>> res) { + this.res = res; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java index fb1b13e7d1e91c..e4d4c3454fe8aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -18,9 +18,10 @@ package org.apache.doris.nereids.rules.exploration.mv; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.MTMV; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.properties.DataTrait; @@ -67,7 +68,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.function.Supplier; @@ -341,10 +341,10 @@ protected Expression tryRewriteExpression(StructInfo queryStructInfo, Expression * compensate union all. */ @Override - protected boolean canUnionRewrite(Plan queryPlan, MTMV mtmv, CascadesContext cascadesContext) { + protected boolean canUnionRewrite(Plan queryPlan, AsyncMaterializationContext context, + CascadesContext cascadesContext) { // Check query plan is contain the partition column // Query plan in the current rule must contain aggregate node, because the rule pattern is - // Optional> logicalAggregateOptional = queryPlan.collectFirst(planTreeNode -> planTreeNode instanceof LogicalAggregate); if (!logicalAggregateOptional.isPresent()) { @@ -355,19 +355,24 @@ protected boolean canUnionRewrite(Plan queryPlan, MTMV mtmv, CascadesContext cas // Scalar aggregate can not compensate union all return false; } - final String relatedCol = mtmv.getMvPartitionInfo().getRelatedCol(); - final BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); + MTMVPartitionInfo mvPartitionInfo = context.getMtmv().getMvPartitionInfo(); + List pctInfos = mvPartitionInfo.getPctInfos(); boolean canUnionRewrite = false; // Check the query plan group by expression contains partition col or not List groupByShuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage(groupByExpressions, queryPlan, new BitSet()); for (Expression expression : groupByShuttledExpressions) { - canUnionRewrite = !expression.collectToSet(expr -> expr instanceof SlotReference - && ((SlotReference) expr).isColumnFromTable() - && Objects.equals(((SlotReference) expr).getOriginalColumn().map(Column::getName).orElse(null), - relatedCol) - && Objects.equals(((SlotReference) expr).getOriginalTable().map(BaseTableInfo::new).orElse(null), - relatedTableInfo)).isEmpty(); + canUnionRewrite = !expression.collectToSet(expr -> { + if (!(expr instanceof SlotReference) || !((SlotReference) expr).isColumnFromTable()) { + return false; + } + String columnName = ((SlotReference) expr).getOriginalColumn().map(Column::getName).orElse(null); + BaseTableInfo baseTableInfo = ((SlotReference) expr).getOriginalTable().map(BaseTableInfo::new) + .orElse(null); + return pctInfos.stream().anyMatch(colInfo -> colInfo.getTableInfo().equals(baseTableInfo) + && colInfo.getColName().equals(columnName)); + } + ).isEmpty(); if (canUnionRewrite) { break; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java index f8a1a6ed2e921d..e4e515fb929b0c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -24,7 +24,9 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.profile.SummaryProfile; import org.apache.doris.common.util.TimeUtils; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.jobs.executor.Rewriter; @@ -201,7 +203,7 @@ protected List getValidQueryStructInfos(Plan queryPlan, CascadesCont * only one materialization every time. Different query pattern should override the sub logic. */ protected List doRewrite(StructInfo queryStructInfo, CascadesContext cascadesContext, - MaterializationContext materializationContext) { + MaterializationContext materializationContext) throws AnalysisException { List rewriteResults = new ArrayList<>(); StructInfo viewStructInfo = materializationContext.getStructInfo(); MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations()); @@ -304,43 +306,45 @@ protected List doRewrite(StructInfo queryStructInfo, CascadesContext casca if (rewrittenPlan == null) { continue; } - Pair>, Map>> invalidPartitions; + Pair>, Map>> invalidPartitions; if (PartitionCompensator.needUnionRewrite(materializationContext) && sessionVariable.isEnableMaterializedViewUnionRewrite()) { MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); - BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); Map, Set> queryUsedPartitions = PartitionCompensator.getQueryUsedPartitions( cascadesContext.getStatementContext(), queryStructInfo.getTableBitSet()); - Set relateTableUsedPartitions = queryUsedPartitions.get(relatedTableInfo.toList()); - if (relateTableUsedPartitions == null) { - materializationContext.recordFailReason(queryStructInfo, - String.format("queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", - relatedTableInfo.toList(), - cascadesContext.getConnectContext().getQueryIdentifier()), - () -> String.format( - "queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", - relatedTableInfo.toList(), - cascadesContext.getConnectContext().getQueryIdentifier())); - LOG.warn(String.format( - "queryUsedPartition is null but needUnionRewrite, table is %s, queryId is %s", - relatedTableInfo.toList(), cascadesContext.getConnectContext().getQueryIdentifier())); - return rewriteResults; + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + boolean relateTableUsedPartitionsAnyNull = false; + boolean relateTableUsedPartitionsAllEmpty = true; + for (MTMVRelatedTableIf tableIf : pctTables) { + Set queryUsedPartition = queryUsedPartitions.get(tableIf.getFullQualifiers()); + relateTableUsedPartitionsAnyNull |= queryUsedPartition == null; + if (queryUsedPartition == null) { + continue; + } + relateTableUsedPartitionsAllEmpty &= queryUsedPartition.isEmpty(); } - if (relateTableUsedPartitions.isEmpty()) { + if (relateTableUsedPartitionsAnyNull || relateTableUsedPartitionsAllEmpty) { materializationContext.recordFailReason(queryStructInfo, - String.format("queryUsedPartition is empty, table is %s, queryId is %s", - relatedTableInfo.toList(), + String.format("queryUsedPartition is all null or empty but needUnionRewrite, " + + "queryUsedPartitions is %s, queryId is %s", + queryUsedPartitions, cascadesContext.getConnectContext().getQueryIdentifier()), - () -> String.format("queryUsedPartition is empty, table is %s, queryId is %s", - relatedTableInfo.toList(), + () -> String.format( + "queryUsedPartition is all null or empty but needUnionRewrite, " + + "queryUsedPartitions is %s, queryId is %s", + queryUsedPartitions, 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 + if (LOG.isDebugEnabled()) { + LOG.debug(String.format( + "queryUsedPartition is all null or empty but needUnionRewrite, " + + "queryUsedPartitions is %s, queryId is %s", + queryUsedPartitions, + cascadesContext.getConnectContext().getQueryIdentifier())); + } return rewriteResults; } try { - invalidPartitions = calcInvalidPartitions(relateTableUsedPartitions, rewrittenPlan, + invalidPartitions = calcInvalidPartitions(queryUsedPartitions, rewrittenPlan, cascadesContext, (AsyncMaterializationContext) materializationContext); } catch (AnalysisException e) { materializationContext.recordFailReason(queryStructInfo, @@ -358,7 +362,8 @@ protected List doRewrite(StructInfo queryStructInfo, CascadesContext casca return rewriteResults; } boolean partitionNeedUnion = PartitionCompensator.needUnionRewrite(invalidPartitions, cascadesContext); - boolean canUnionRewrite = canUnionRewrite(queryPlan, mtmv, cascadesContext); + boolean canUnionRewrite = canUnionRewrite(queryPlan, + (AsyncMaterializationContext) materializationContext, cascadesContext); if (partitionNeedUnion && !canUnionRewrite) { materializationContext.recordFailReason(queryStructInfo, "need compensate union all, but can not, because the query structInfo", @@ -366,21 +371,17 @@ protected List doRewrite(StructInfo queryStructInfo, CascadesContext casca mtmv.getMvPartitionInfo(), queryPlan.treeString())); return rewriteResults; } - final Pair>, Map>> finalInvalidPartitions = - invalidPartitions; if (partitionNeedUnion) { Pair planAndNeedAddFilterPair = - StructInfo.addFilterOnTableScan(queryPlan, invalidPartitions.value(), - mtmv.getMvPartitionInfo().getRelatedCol(), cascadesContext); + StructInfo.addFilterOnTableScan(queryPlan, invalidPartitions.value(), cascadesContext); if (planAndNeedAddFilterPair == null) { materializationContext.recordFailReason(queryStructInfo, "Add filter to base table fail when union rewrite", - () -> String.format("invalidPartitions are %s, queryPlan is %s, partition column is %s", - invalidPartitions, queryPlan.treeString(), - mtmv.getMvPartitionInfo().getPartitionCol())); + () -> String.format("invalidPartitions are %s, queryPlan is %s", + invalidPartitions, queryPlan.treeString())); continue; } - if (finalInvalidPartitions.value().isEmpty() || !planAndNeedAddFilterPair.value()) { + if (invalidPartitions.value().isEmpty() || !planAndNeedAddFilterPair.value()) { // if invalid base table filter is empty or doesn't need to add filter on base table, // only need remove mv invalid partition rewrittenPlan = rewrittenPlan.accept(new PartitionRemover(), invalidPartitions.key()); @@ -478,7 +479,8 @@ private static void trySetStatistics(MaterializationContext context, CascadesCon * If mv part partition is invalid, can not compensate union all, because result is wrong after * compensate union all. */ - protected boolean canUnionRewrite(Plan queryPlan, MTMV mtmv, CascadesContext cascadesContext) { + protected boolean canUnionRewrite(Plan queryPlan, AsyncMaterializationContext context, + CascadesContext cascadesContext) { return true; } @@ -501,13 +503,13 @@ protected boolean isOutputValid(Plan sourcePlan, Plan rewrittenPlan) { * So we should calc the invalid partition used in query * @return the key in pair is mvNeedRemovePartitionNameSet, the value in pair is baseTableNeedUnionPartitionNameSet */ - protected Pair>, Map>> calcInvalidPartitions( - Set queryUsedPartition, + protected Pair>, Map>> calcInvalidPartitions( + Map, Set> queryUsedBaseTablePartitionMap, Plan rewrittenPlan, CascadesContext cascadesContext, AsyncMaterializationContext materializationContext) throws AnalysisException { - return PartitionCompensator.calcInvalidPartitions(queryUsedPartition, rewrittenPlan, + return PartitionCompensator.calcInvalidPartitions(queryUsedBaseTablePartitionMap, rewrittenPlan, materializationContext, cascadesContext); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java index 13ed940350effc..20d28036fcb1bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AsyncMaterializationContext.java @@ -19,9 +19,11 @@ import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.Table; +import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Id; import org.apache.doris.common.Pair; import org.apache.doris.mtmv.MTMVCache; +import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.rules.exploration.mv.mapping.ExpressionMapping; import org.apache.doris.nereids.trees.plans.ObjectId; @@ -39,9 +41,12 @@ 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.Optional; +import java.util.Set; /** * Async context for query rewrite by materialized view @@ -50,6 +55,7 @@ public class AsyncMaterializationContext extends MaterializationContext { private static final Logger LOG = LogManager.getLogger(AsyncMaterializationContext.class); private final MTMV mtmv; + private Map>> partitionMultiFlatMap; /** * MaterializationContext, this contains necessary info for query rewriting by mv @@ -174,4 +180,28 @@ public StructInfo getStructInfo() { public boolean isSuccess() { return success; } + + /** + * Calculate partition mappings and cache + */ + public Map>> calculatePartitionMappings() throws AnalysisException { + if (partitionMultiFlatMap != null) { + return partitionMultiFlatMap; + } + partitionMultiFlatMap = new HashMap<>(); + Map>> partitionMultiMap = this.mtmv.calculatePartitionMappings(); + for (Map.Entry>> entry : partitionMultiMap.entrySet()) { + String partitionKey = entry.getKey(); + Map> tableMap = entry.getValue(); + for (Map.Entry> tableEntry : tableMap.entrySet()) { + MTMVRelatedTableIf table = tableEntry.getKey(); + Set set = tableEntry.getValue(); + partitionMultiFlatMap + .computeIfAbsent(table, k -> new HashMap<>()) + .computeIfAbsent(partitionKey, k -> new HashSet<>()) + .addAll(set); + } + } + return partitionMultiFlatMap; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java index 06e1e77e56bc33..813136b067d7fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateOnNoneAggregateRule.java @@ -19,6 +19,7 @@ import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.rules.Rule; @@ -91,14 +92,14 @@ protected boolean checkMaterializationPattern(StructInfo structInfo, CascadesCon } @Override - protected Pair>, Map>> calcInvalidPartitions( - Set queryUsedPartition, + protected Pair>, Map>> calcInvalidPartitions( + Map, Set> queryUsedBaseTablePartitionMap, Plan rewrittenPlan, CascadesContext cascadesContext, AsyncMaterializationContext materializationContext) throws AnalysisException { - Pair>, Map>> invalidPartitions - = super.calcInvalidPartitions(queryUsedPartition, rewrittenPlan, cascadesContext, + Pair>, Map>> invalidPartitions + = super.calcInvalidPartitions(queryUsedBaseTablePartitionMap, rewrittenPlan, cascadesContext, materializationContext); if (PartitionCompensator.needUnionRewrite(invalidPartitions, cascadesContext)) { // if query use some invalid partition in mv, bail out diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java index a8da059b364fc4..d92636b87f78ae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtils.java @@ -17,18 +17,8 @@ package org.apache.doris.nereids.rules.exploration.mv; -import org.apache.doris.analysis.Expr; -import org.apache.doris.analysis.SlotRef; -import org.apache.doris.catalog.Column; import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.PartitionType; -import org.apache.doris.catalog.TableIf; -import org.apache.doris.catalog.constraint.TableIdentifier; -import org.apache.doris.common.DdlException; import org.apache.doris.common.Pair; -import org.apache.doris.datasource.mvcc.MvccUtil; -import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.PlannerHook; import org.apache.doris.nereids.StatementContext; @@ -36,52 +26,38 @@ import org.apache.doris.nereids.memo.StructInfoMap; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.BindRelation; -import org.apache.doris.nereids.rules.expression.ExpressionNormalization; -import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.rules.exploration.mv.PartitionIncrementMaintainer.PartitionIncrementCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.PartitionIncrementMaintainer.PartitionIncrementChecker; +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo.RelatedTableColumnInfo; import org.apache.doris.nereids.rules.rewrite.QueryPartitionCollector; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.functions.scalar.NonNullable; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nullable; -import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; -import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PreAggStatus; -import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.algebra.Sink; -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.LogicalCTEProducer; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; -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.LogicalLimit; 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.LogicalRelation; -import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; -import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.nereids.trees.plans.visitor.NondeterministicFunctionCollector; -import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.qe.SessionVariable; -import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; -import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.BitSet; @@ -108,9 +84,9 @@ public class MaterializedViewUtils { * @param materializedViewPlan this should be rewritten or analyzed plan, should not be physical plan. * @param column ref column name. */ + @Deprecated public static RelatedTableInfo getRelatedTableInfo(String column, String timeUnit, Plan materializedViewPlan, CascadesContext cascadesContext) { - List outputExpressions = materializedViewPlan.getOutput(); NamedExpression columnExpr = null; // get column slot @@ -123,34 +99,122 @@ public static RelatedTableInfo getRelatedTableInfo(String column, String timeUni if (columnExpr == null) { return RelatedTableInfo.failWith("partition column can not find from sql select column"); } + materializedViewPlan = PartitionIncrementMaintainer.removeSink(materializedViewPlan); + Expression dateTrunc = null; if (timeUnit != null) { - Expression dateTrunc = new DateTrunc(columnExpr, new VarcharLiteral(timeUnit)); + dateTrunc = new DateTrunc(columnExpr, new VarcharLiteral(timeUnit)); columnExpr = new Alias(dateTrunc); materializedViewPlan = new LogicalProject<>(ImmutableList.of(columnExpr), materializedViewPlan); } - // Collect table relation map which is used to identify self join - List catalogRelations = materializedViewPlan.collectToList(CatalogRelation.class::isInstance); - ImmutableMultimap.Builder tableCatalogRelationMultimapBuilder = - ImmutableMultimap.builder(); - for (CatalogRelation catalogRelation : catalogRelations) { - tableCatalogRelationMultimapBuilder.put(new TableIdentifier(catalogRelation.getTable()), catalogRelation); - } // Check sql pattern - IncrementCheckerContext checkContext = - new IncrementCheckerContext(columnExpr, tableCatalogRelationMultimapBuilder.build(), cascadesContext); - materializedViewPlan.accept(MaterializedViewIncrementChecker.INSTANCE, checkContext); - Multimap partitionRelatedTableAndColumnMap = - checkContext.getPartitionRelatedTableAndColumnMap(); - if (partitionRelatedTableAndColumnMap.isEmpty()) { - return RelatedTableInfo.failWith(String.format("can't not find valid partition track column, because %s", - String.join(",", checkContext.getFailReasons()))); + Map producerCteIdToPlanMap = collectProducerCtePlans(materializedViewPlan); + PartitionIncrementCheckContext checkContext = new PartitionIncrementCheckContext( + columnExpr, dateTrunc, producerCteIdToPlanMap, materializedViewPlan, cascadesContext); + checkContext.getPartitionAndRefExpressionMap().put(columnExpr, + RelatedTableColumnInfo.of(columnExpr, null, true, false)); + materializedViewPlan.accept(PartitionIncrementChecker.INSTANCE, checkContext); + List checkedTableColumnInfos = + PartitionIncrementMaintainer.getRelatedTableColumnInfosWithCheck(checkContext, tableColumnInfo -> + tableColumnInfo.isOriginalPartition() && tableColumnInfo.isFromTablePartitionColumn()); + if (checkedTableColumnInfos == null) { + return RelatedTableInfo.failWith("multi partition column data types are different"); + } + if (checkContext.isFailFast()) { + return RelatedTableInfo.failWith("partition column is not in group by or window partition by, " + + checkContext.getFailReasons()); + } + if (!checkedTableColumnInfos.isEmpty()) { + return RelatedTableInfo.successWith(checkedTableColumnInfos); + } + return RelatedTableInfo.failWith(String.format("can't not find valid partition track column, because %s", + String.join(",", checkContext.getFailReasons()))); + } + + /** + * Get related base table info which materialized view plan column reference, + * input param plan should be rewritten plan that sub query should be eliminated + * + * @param materializedViewPlan this should be rewritten or analyzed plan, should not be physical plan. + * @param column ref column name. + */ + public static RelatedTableInfo getRelatedTableInfos(String column, String timeUnit, + Plan materializedViewPlan, CascadesContext cascadesContext) { + List outputExpressions = materializedViewPlan.getOutput(); + NamedExpression columnExpr = null; + // get column slot + for (Slot outputSlot : outputExpressions) { + if (outputSlot.getName().equalsIgnoreCase(column)) { + columnExpr = outputSlot; + break; + } } - // TODO support to return only one related table info, support multi later - for (Map.Entry entry : partitionRelatedTableAndColumnMap.entries()) { - return RelatedTableInfo.successWith(new BaseTableInfo(entry.getKey()), true, - entry.getValue().getName(), checkContext.getPartitionExpression().orElseGet(() -> null)); + if (columnExpr == null) { + return RelatedTableInfo.failWith("partition column can not find from sql select column"); } - return RelatedTableInfo.failWith("can't not find valid partition track column finally"); + materializedViewPlan = PartitionIncrementMaintainer.removeSink(materializedViewPlan); + Expression dateTrunc = null; + if (timeUnit != null) { + dateTrunc = new DateTrunc(columnExpr, new VarcharLiteral(timeUnit)); + columnExpr = new Alias(dateTrunc); + materializedViewPlan = new LogicalProject<>(ImmutableList.of(columnExpr), materializedViewPlan); + } + // Check sql pattern + Map producerCteIdToPlanMap = collectProducerCtePlans(materializedViewPlan); + PartitionIncrementCheckContext checkContext = new PartitionIncrementCheckContext( + columnExpr, dateTrunc, producerCteIdToPlanMap, materializedViewPlan, cascadesContext); + checkContext.getPartitionAndRefExpressionMap().put(columnExpr, + RelatedTableColumnInfo.of(columnExpr, null, true, false)); + materializedViewPlan.accept(PartitionIncrementChecker.INSTANCE, checkContext); + if (!checkPartitionRefExpression(checkContext.getPartitionAndRefExpressionMap().values())) { + return RelatedTableInfo.failWith(String.format( + "partition ref expressions is not consistent, partition ref expressions map is %s", + checkContext.getPartitionAndRefExpressionMap())); + } + List checkedTableColumnInfos = + PartitionIncrementMaintainer.getRelatedTableColumnInfosWithCheck(checkContext, + RelatedTableColumnInfo::isFromTablePartitionColumn); + if (checkedTableColumnInfos == null) { + return RelatedTableInfo.failWith("multi partition column data types are different"); + } + if (checkContext.isFailFast()) { + return RelatedTableInfo.failWith("partition column is not in group by or window partition by, " + + checkContext.getFailReasons()); + } + if (!checkedTableColumnInfos.isEmpty()) { + return RelatedTableInfo.successWith(checkedTableColumnInfos); + } + return RelatedTableInfo.failWith(String.format("can't not find valid partition track column, because %s", + String.join(",", checkContext.getFailReasons()))); + } + + private static Map collectProducerCtePlans(Plan plan) { + Map collectProducerCtePlans = new HashMap<>(); + plan.accept(new DefaultPlanVisitor>() { + @Override + public Void visitLogicalCTEProducer(LogicalCTEProducer cteProducer, + Map context) { + context.put(cteProducer.getCteId(), cteProducer); + return super.visitLogicalCTEProducer(cteProducer, context); + } + }, collectProducerCtePlans); + return collectProducerCtePlans; + } + + /** + * Check the partition expression date_trunc num is valid or not + */ + private static boolean checkPartitionRefExpression(Collection refExpressions) { + for (RelatedTableColumnInfo tableColumnInfo : refExpressions) { + if (tableColumnInfo.getPartitionExpression().isPresent()) { + // If partition ref up expression is empty, return false directly + List dateTruncs = + tableColumnInfo.getPartitionExpression().get().collectToList(DateTrunc.class::isInstance); + if (dateTruncs.size() > 1) { + return false; + } + } + } + return true; } /** @@ -526,429 +590,4 @@ public Boolean visit(Plan plan, Void context) { return false; } } - - private static final class MaterializedViewIncrementChecker extends - DefaultPlanVisitor { - - public static final MaterializedViewIncrementChecker INSTANCE = new MaterializedViewIncrementChecker(); - public static final Set> SUPPORT_EXPRESSION_TYPES = - ImmutableSet.of(DateTrunc.class, SlotReference.class, Literal.class); - - @Override - public Void visitLogicalProject(LogicalProject project, IncrementCheckerContext context) { - List output = project.getOutput(); - boolean isValid = checkPartition(output, project, context); - if (!isValid) { - return null; - } - return visit(project, context); - } - - @Override - public Void visitLogicalFilter(LogicalFilter filter, IncrementCheckerContext context) { - return visit(filter, context); - } - - @Override - public Void visitLogicalJoin(LogicalJoin join, - IncrementCheckerContext context) { - if (join.isMarkJoin()) { - context.addFailReason("partition track doesn't support mark join"); - return null; - } - Plan left = join.child(0); - Set leftColumnSet = left.getOutputSet().stream() - .filter(slot -> slot instanceof SlotReference - && slot.isColumnFromTable()) - .map(slot -> ((SlotReference) slot).getOriginalColumn().get()) - .collect(Collectors.toSet()); - SlotReference contextPartitionColumn = getContextPartitionColumn(context); - if (contextPartitionColumn == null) { - return null; - } - boolean useLeft = leftColumnSet.contains(contextPartitionColumn.getOriginalColumn().get()); - JoinType joinType = join.getJoinType(); - if (joinType.isInnerJoin() || joinType.isCrossJoin()) { - return visit(join, context); - } else if ((joinType.isLeftJoin() - || joinType.isLeftSemiJoin() - || joinType.isLeftAntiJoin()) && useLeft) { - return join.left().accept(this, context); - } else if ((joinType.isRightJoin() - || joinType.isRightAntiJoin() - || joinType.isRightSemiJoin()) && !useLeft) { - return join.right().accept(this, context); - } - context.addFailReason(String.format("partition column is in un supported join null generate side, " - + "current join type is %s", joinType)); - return null; - } - - @Override - public Void visitLogicalRelation(LogicalRelation relation, IncrementCheckerContext context) { - if (!(relation instanceof LogicalCatalogRelation)) { - context.addFailReason(String.format("relation should be LogicalCatalogRelation, " - + "but now is %s", relation.getClass().getSimpleName())); - return null; - } - SlotReference contextPartitionColumn = getContextPartitionColumn(context); - if (contextPartitionColumn == null) { - context.addFailReason(String.format("mv partition column is not from table when relation check, " - + "mv partition column is %s", context.getMvPartitionColumn())); - return null; - } - // Check the table which mv partition column belonged to is same as the current check relation or not - if (!((LogicalCatalogRelation) relation).getTable().getFullQualifiers().equals( - contextPartitionColumn.getOriginalTable() - .map(TableIf::getFullQualifiers).orElse(ImmutableList.of()))) { - context.addFailReason(String.format("mv partition column name is not belonged to current check , " - + "table, current table is %s", - ((LogicalCatalogRelation) relation).getTable().getFullQualifiers())); - return null; - } - LogicalCatalogRelation logicalCatalogRelation = (LogicalCatalogRelation) relation; - TableIf table = logicalCatalogRelation.getTable(); - // if self join, self join can not partition track now, remove the partition column correspondingly - if (context.getRelationByTable(table).size() > 1) { - context.getPartitionRelatedTableAndColumnMap().removeAll(table); - context.addFailReason(String.format("self join doesn't support partition update, " - + "self join table name is %s", table.getName())); - return null; - } - // TODO: 2024/1/31 support only one partition referenced column, support multi later - if (!context.getPartitionRelatedTableAndColumnMap().isEmpty()) { - context.addFailReason(String.format("partition track already has an related base table column," - + "track info is %s", context.getPartitionRelatedTableAndColumnMap())); - return null; - } - if (!(table instanceof MTMVRelatedTableIf)) { - context.addFailReason(String.format("relation base table is not MTMVRelatedTableIf, the table is %s", - table.getName())); - return null; - } - MTMVRelatedTableIf relatedTable = (MTMVRelatedTableIf) table; - PartitionType type = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); - if (PartitionType.UNPARTITIONED.equals(type)) { - context.addFailReason(String.format("related base table is not partition table, the table is %s", - table.getName())); - return null; - } - Set partitionColumnSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - try { - partitionColumnSet.addAll(relatedTable.getPartitionColumnNames( - MvccUtil.getSnapshotFromContext(relatedTable))); - } catch (DdlException e) { - context.addFailReason(e.getMessage()); - return null; - } - Column mvReferenceColumn = contextPartitionColumn.getOriginalColumn().get(); - Expr definExpr = mvReferenceColumn.getDefineExpr(); - if (definExpr instanceof SlotRef) { - Column referenceRollupColumn = ((SlotRef) definExpr).getColumn(); - if (referenceRollupColumn != null) { - mvReferenceColumn = referenceRollupColumn; - } - } - if (partitionColumnSet.contains(mvReferenceColumn.getName()) - && (!mvReferenceColumn.isAllowNull() || relatedTable.isPartitionColumnAllowNull())) { - context.addTableColumn(table, mvReferenceColumn); - } else { - context.addFailReason(String.format("related base table partition column doesn't contain the mv" - + " partition or partition nullable check fail, the mvReferenceColumn is %s", - mvReferenceColumn)); - } - return visit(relation, context); - } - - @Override - public Void visitLogicalAggregate(LogicalAggregate aggregate, - IncrementCheckerContext context) { - Set groupByExprSet = new HashSet<>(aggregate.getGroupByExpressions()); - if (groupByExprSet.isEmpty()) { - context.addFailReason("group by sets is empty, doesn't contain the target partition"); - return null; - } - boolean isValid = checkPartition(groupByExprSet, aggregate, context); - if (!isValid) { - return null; - } - return visit(aggregate, context); - } - - @Override - public Void visitLogicalWindow(LogicalWindow window, IncrementCheckerContext context) { - List windowExpressions = window.getWindowExpressions(); - if (windowExpressions.isEmpty()) { - return visit(window, context); - } - for (NamedExpression namedExpression : windowExpressions) { - if (!checkWindowPartition(namedExpression, context)) { - context.addFailReason("window partition sets doesn't contain the target partition"); - return null; - } - } - return super.visitLogicalWindow(window, context); - } - - @Override - public Void visit(Plan plan, IncrementCheckerContext context) { - if (plan instanceof LogicalProject - || plan instanceof LogicalLimit - || plan instanceof LogicalFilter - || plan instanceof LogicalJoin - || plan instanceof LogicalAggregate - || plan instanceof LogicalCatalogRelation - || plan instanceof LogicalResultSink - || plan instanceof LogicalWindow) { - return super.visit(plan, context); - } - context.addFailReason(String.format("Unsupported plan operate in track partition, " - + "the invalid plan node is %s", plan.getClass().getSimpleName())); - return null; - } - - private boolean checkWindowPartition(Expression expression, IncrementCheckerContext context) { - List windowExpressions = - expression.collectToList(expressionTreeNode -> expressionTreeNode instanceof WindowExpression); - for (Object windowExpressionObj : windowExpressions) { - WindowExpression windowExpression = (WindowExpression) windowExpressionObj; - List partitionKeys = windowExpression.getPartitionKeys(); - Set originalPartitionbyExprSet = new HashSet<>(); - partitionKeys.forEach(groupExpr -> { - if (groupExpr instanceof SlotReference && groupExpr.isColumnFromTable()) { - originalPartitionbyExprSet.add(((SlotReference) groupExpr).getOriginalColumn().get()); - } - }); - SlotReference contextPartitionColumn = getContextPartitionColumn(context); - if (contextPartitionColumn == null) { - return false; - } - if (!originalPartitionbyExprSet.contains(contextPartitionColumn.getOriginalColumn().get())) { - return false; - } - } - return true; - } - - private SlotReference getContextPartitionColumn(IncrementCheckerContext context) { - if (!context.getMvPartitionColumn().isColumnFromTable()) { - context.addFailReason(String.format("context partition column should be slot from column, " - + "context column is %s", - context.getMvPartitionColumn())); - return null; - } - return (SlotReference) context.getMvPartitionColumn(); - } - - /** - * Given a partition named expression and expressionsToCheck, check the partition is valid - * example 1: - * partition expression is date_trunc(date_alias#25, 'hour') AS `date_trunc(date_alias, 'hour')`#30 - * expressionsToCheck is date_trunc(date_alias, 'hour')#30 - * expressionsToCheck is the slot to partition expression, but they are expression - * example 2: - * partition expression is L_SHIPDATE#10 - * expressionsToCheck isL_SHIPDATE#10 - * both of them are slot - * example 3: - * partition expression is date_trunc(L_SHIPDATE#10, 'hour')#30 - * expressionsToCheck is L_SHIPDATE#10 - * all above should check successfully - * */ - private static boolean checkPartition(Collection expressionsToCheck, Plan plan, - IncrementCheckerContext context) { - NamedExpression partitionColumn = context.getMvPartitionColumn(); - - OUTER_CHECK: for (Expression projectSlot : expressionsToCheck) { - if (projectSlot.isColumnFromTable() && projectSlot.equals(partitionColumn.toSlot())) { - continue; - } - // check the expression which use partition column - Expression expressionToCheck = - ExpressionUtils.shuttleExpressionWithLineage(projectSlot, plan, new BitSet()); - // merge date_trunc - expressionToCheck = new ExpressionNormalization().rewrite(expressionToCheck, - new ExpressionRewriteContext(context.getCascadesContext())); - - Expression partitionExpression = context.getPartitionExpression().isPresent() - ? context.getPartitionExpression().get() : - ExpressionUtils.shuttleExpressionWithLineage(partitionColumn, plan, new BitSet()); - // merge date_trunc - partitionExpression = new ExpressionNormalization().rewrite(partitionExpression, - new ExpressionRewriteContext(context.getCascadesContext())); - - Set expressionToCheckColumns = - expressionToCheck.collectToSet(SlotReference.class::isInstance); - Set partitionColumns = - partitionExpression.collectToSet(SlotReference.class::isInstance); - if (Sets.intersection(expressionToCheckColumns, partitionColumns).isEmpty() - || expressionToCheckColumns.isEmpty() || partitionColumns.isEmpty()) { - // this expression doesn't use partition column - continue; - } - if (expressionToCheckColumns.size() > 1 || partitionColumns.size() > 1) { - context.addFailReason( - String.format("partition expression use more than one slot reference, invalid " - + "expressionToCheckColumns is %s, partitionColumnDateColumns is %s", - expressionToCheckColumns, partitionColumns)); - continue; - } - List expressions = expressionToCheck.collectToList(Expression.class::isInstance); - for (Expression expression : expressions) { - if (SUPPORT_EXPRESSION_TYPES.stream().noneMatch( - supportExpression -> supportExpression.isAssignableFrom(expression.getClass()))) { - context.addFailReason( - String.format("column to check use invalid implicit expression, invalid " - + "expression is %s", expression)); - continue OUTER_CHECK; - } - } - List partitionExpressions = partitionExpression.collectToList( - Expression.class::isInstance); - for (Expression expression : partitionExpressions) { - if (SUPPORT_EXPRESSION_TYPES.stream().noneMatch( - supportExpression -> supportExpression.isAssignableFrom(expression.getClass()))) { - context.addFailReason( - String.format("partition column use invalid implicit expression, invalid " - + "expression is %s", expression)); - continue OUTER_CHECK; - } - } - List expressionToCheckDataTruncList = - expressionToCheck.collectToList(DateTrunc.class::isInstance); - List partitionColumnDateTrucList = - partitionExpression.collectToList(DateTrunc.class::isInstance); - if (expressionToCheckDataTruncList.size() > 1 || partitionColumnDateTrucList.size() > 1) { - // mv time unit level is little then query - context.addFailReason("partition column time unit level should be " - + "greater than sql select column"); - continue; - } - if (!partitionColumn.isColumnFromTable()) { - context.setMvPartitionColumn(partitionColumns.iterator().next()); - } - if (!context.getPartitionExpression().isPresent()) { - context.setPartitionExpression(partitionExpression); - } - return true; - } - return context.getMvPartitionColumn().isColumnFromTable(); - } - } - - private static final class IncrementCheckerContext { - private NamedExpression mvPartitionColumn; - private Optional partitionExpression = Optional.empty(); - private final Multimap tableAndCatalogRelationMap; - private final Multimap partitionRelatedTableAndColumnMap = HashMultimap.create(); - private final Set failReasons = new HashSet<>(); - private final CascadesContext cascadesContext; - - public IncrementCheckerContext(NamedExpression mvPartitionColumn, - Multimap tableAndCatalogRelationMap, - CascadesContext cascadesContext) { - this.mvPartitionColumn = mvPartitionColumn; - this.tableAndCatalogRelationMap = tableAndCatalogRelationMap; - this.cascadesContext = cascadesContext; - } - - public NamedExpression getMvPartitionColumn() { - return mvPartitionColumn; - } - - public void setMvPartitionColumn(NamedExpression mvPartitionColumn) { - this.mvPartitionColumn = mvPartitionColumn; - } - - public void addTableColumn(TableIf relatedTable, Column partitionColumn) { - partitionRelatedTableAndColumnMap.put(relatedTable, partitionColumn); - } - - public Multimap getPartitionRelatedTableAndColumnMap() { - return partitionRelatedTableAndColumnMap; - } - - public Collection getRelationByTable(TableIf tableIf) { - return tableAndCatalogRelationMap.get(new TableIdentifier(tableIf)); - } - - public void addTableAndRelation(TableIf tableIf, CatalogRelation relation) { - tableAndCatalogRelationMap.put(new TableIdentifier(tableIf), relation); - } - - public Set getFailReasons() { - return failReasons; - } - - public void addFailReason(String failReason) { - this.failReasons.add(failReason); - } - - public CascadesContext getCascadesContext() { - return cascadesContext; - } - - public Optional getPartitionExpression() { - return partitionExpression; - } - - public void setPartitionExpression(Expression partitionExpression) { - this.partitionExpression = Optional.ofNullable(partitionExpression); - } - } - - /** - * The related table info that mv relate - */ - public static final class RelatedTableInfo { - private final BaseTableInfo tableInfo; - private final boolean pctPossible; - private final String column; - private final Set failReasons = new HashSet<>(); - // This records the partition expression if exist - private final Optional partitionExpression; - - public RelatedTableInfo(BaseTableInfo tableInfo, boolean pctPossible, String column, String failReason, - Expression partitionExpression) { - this.tableInfo = tableInfo; - this.pctPossible = pctPossible; - this.column = column; - this.failReasons.add(failReason); - this.partitionExpression = Optional.ofNullable(partitionExpression); - } - - public static RelatedTableInfo failWith(String failReason) { - return new RelatedTableInfo(null, false, null, failReason, - null); - } - - public static RelatedTableInfo successWith(BaseTableInfo tableInfo, boolean pctPossible, String column, - Expression partitionExpression) { - return new RelatedTableInfo(tableInfo, pctPossible, column, "", partitionExpression); - } - - public BaseTableInfo getTableInfo() { - return tableInfo; - } - - public boolean isPctPossible() { - return pctPossible; - } - - public String getColumn() { - return column; - } - - public void addFailReason(String failReason) { - this.failReasons.add(failReason); - } - - public String getFailReason() { - return String.join(",", failReasons); - } - - public Optional getPartitionExpression() { - return partitionExpression; - } - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java index 0ee88ce82af070..7cc649a68b0148 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionCompensator.java @@ -24,8 +24,9 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.mtmv.MTMVPartitionInfo; +import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.Plan; @@ -35,6 +36,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import org.apache.logging.log4j.LogManager; @@ -64,18 +66,16 @@ public class PartitionCompensator { /** * 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 queryUsedBaseTablePartitionMap partitions used by query related partition table * @param rewrittenPlan tmp rewrittenPlan when mv rewrite * @param materializationContext the context of materialization,which hold materialized view meta and other info * @param cascadesContext the context of cascades * @return the key in pair is mvNeedRemovePartitionNameSet, the value in pair is baseTableNeedUnionPartitionNameSet */ - public static Pair>, Map>> calcInvalidPartitions( - Set queryUsedBaseTablePartitionNameSet, Plan rewrittenPlan, + public static Pair>, Map>> calcInvalidPartitions( + Map, Set> queryUsedBaseTablePartitionMap, Plan rewrittenPlan, AsyncMaterializationContext materializationContext, CascadesContext cascadesContext) throws AnalysisException { - Set mvNeedRemovePartitionNameSet = new HashSet<>(); - Set baseTableNeedUnionPartitionNameSet = new HashSet<>(); // check partition is valid or not MTMV mtmv = materializationContext.getMtmv(); PartitionInfo mvPartitionInfo = mtmv.getPartitionInfo(); @@ -83,70 +83,142 @@ public static Pair>, Map mvValidPartitions = cascadesContext.getStatementContext() .getMvCanRewritePartitionsMap().get(new BaseTableInfo(mtmv)); + Set rewrittenPlanUsePartitionNameSet = new HashSet<>(); + List mvOlapScanList = rewrittenPlan.collectToList(node -> + node instanceof LogicalOlapScan + && Objects.equals(((CatalogRelation) node).getTable().getName(), mtmv.getName())); + for (LogicalOlapScan olapScan : mvOlapScanList) { + olapScan.getSelectedPartitionIds().forEach(id -> + rewrittenPlanUsePartitionNameSet.add(olapScan.getTable().getPartition(id).getName())); + } + Map>> mtmvRelatedTableIfMapMap + = materializationContext.calculatePartitionMappings(); + boolean allCompensateIsNull = true; + + Map> mvPartitionNeedRemoveNameMap = new HashMap<>(); + Map> baseTablePartitionNeedUnionNameMap = new HashMap<>(); + + Map pctInfoMap = new HashMap<>(); + mtmv.getMvPartitionInfo().getPctInfos().forEach( + colInfo -> pctInfoMap.put(colInfo.getTableInfo(), colInfo)); + + for (Map.Entry>> partitionMapping + : mtmvRelatedTableIfMapMap.entrySet()) { + MTMVRelatedTableIf relatedTable = partitionMapping.getKey(); + Set relatedTableUsedPartitionSet + = queryUsedBaseTablePartitionMap.get(relatedTable.getFullQualifiers()); + Pair>, Pair>> needCompensatePartitions + = getNeedCompensatePartitions(mvValidPartitions, relatedTableUsedPartitionSet, + rewrittenPlanUsePartitionNameSet, pctInfoMap.get(new BaseTableInfo(relatedTable)), + partitionMapping.getValue(), materializationContext); + allCompensateIsNull &= needCompensatePartitions == null; + if (needCompensatePartitions == null) { + continue; + } + Pair> mvNeedRemovePartition = needCompensatePartitions.key(); + Pair> baseTableNeedUnionTable = needCompensatePartitions.value(); + if ((mvNeedRemovePartition.value().isEmpty() && baseTableNeedUnionTable.value().isEmpty())) { + continue; + } + if (!mvNeedRemovePartition.value().isEmpty()) { + mvPartitionNeedRemoveNameMap + .computeIfAbsent(mvNeedRemovePartition.key(), k -> new HashSet<>()) + .addAll(mvNeedRemovePartition.value()); + } + if (!baseTableNeedUnionTable.value().isEmpty()) { + baseTablePartitionNeedUnionNameMap + .computeIfAbsent(baseTableNeedUnionTable.key(), k -> new HashSet<>()) + .addAll(baseTableNeedUnionTable.value()); + } + // merge all partition to delete or union + Set needRemovePartitionSet = new HashSet<>(); + mvPartitionNeedRemoveNameMap.values().forEach(needRemovePartitionSet::addAll); + mvPartitionNeedRemoveNameMap.replaceAll((k, v) -> needRemovePartitionSet); + + // consider multi base table partition name not same, how to handle it? + Set needUnionPartitionSet = new HashSet<>(); + baseTablePartitionNeedUnionNameMap.values().forEach(needUnionPartitionSet::addAll); + baseTablePartitionNeedUnionNameMap.replaceAll((k, v) -> needUnionPartitionSet); + } + if (allCompensateIsNull) { + return null; + } + return Pair.of(mvPartitionNeedRemoveNameMap, baseTablePartitionNeedUnionNameMap); + } + + private static Pair>, Pair>> getNeedCompensatePartitions( + Collection mvValidPartitions, + Set queryUsedBaseTablePartitionNameSet, + Set rewrittenPlanUsePartitionNameSet, + BaseColInfo relatedPartitionTable, + Map> partitionMapping, + MaterializationContext materializationContext + ) { + // compensated result + Set baseTableNeedUnionPartitionNameSet = new HashSet<>(); + // the middle result when compensate Set mvValidPartitionNameSet = new HashSet<>(); Set mvValidBaseTablePartitionNameSet = new HashSet<>(); Set mvValidHasDataRelatedBaseTableNameSet = new HashSet<>(); - Pair>, Map> partitionMapping = mtmv.calculateDoublyPartitionMappings(); + MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); for (Partition mvValidPartition : mvValidPartitions) { mvValidPartitionNameSet.add(mvValidPartition.getName()); - Set relatedBaseTablePartitions = partitionMapping.key().get(mvValidPartition.getName()); + Set relatedBaseTablePartitions = partitionMapping.get(mvValidPartition.getName()); if (relatedBaseTablePartitions != null) { mvValidBaseTablePartitionNameSet.addAll(relatedBaseTablePartitions); - } - if (!mtmv.selectNonEmptyPartitionIds(ImmutableList.of(mvValidPartition.getId())).isEmpty()) { - if (relatedBaseTablePartitions != null) { + if (!mtmv.selectNonEmptyPartitionIds(ImmutableList.of(mvValidPartition.getId())).isEmpty()) { mvValidHasDataRelatedBaseTableNameSet.addAll(relatedBaseTablePartitions); } } } if (Sets.intersection(mvValidHasDataRelatedBaseTableNameSet, queryUsedBaseTablePartitionNameSet).isEmpty()) { - // if mv can not offer any partition for query, query rewrite bail out + // if mv couldn't offer any partition for query, query rewrite should bail out return null; } - // Check when mv partition relates base table partition data change or delete partition - Set rewrittenPlanUsePartitionNameSet = new HashSet<>(); - List mvOlapScanList = rewrittenPlan.collectToList(node -> - node instanceof LogicalOlapScan - && Objects.equals(((CatalogRelation) node).getTable().getName(), mtmv.getName())); - for (Object olapScanObj : mvOlapScanList) { - LogicalOlapScan olapScan = (LogicalOlapScan) olapScanObj; - olapScan.getSelectedPartitionIds().forEach(id -> - rewrittenPlanUsePartitionNameSet.add(olapScan.getTable().getPartition(id).getName())); - } - // If rewritten plan use but not in mv valid partition name set, need remove in mv and base table union + // Check when mv partition relates base table partition data change or delete partition, + // the mv partition would be invalid. + // Partitions rewritten plan used but not in mv valid partition name set, + // need to be removed in mv and union base table + Set mvNeedRemovePartitionNameSet = new HashSet<>(); Sets.difference(rewrittenPlanUsePartitionNameSet, mvValidPartitionNameSet) .copyInto(mvNeedRemovePartitionNameSet); for (String partitionName : mvNeedRemovePartitionNameSet) { - baseTableNeedUnionPartitionNameSet.addAll(partitionMapping.key().get(partitionName)); + Set baseTablePartitions = partitionMapping.get(partitionName); + if (baseTablePartitions == null) { + // Base table partition maybe deleted, need not union + continue; + } + baseTableNeedUnionPartitionNameSet.addAll(baseTablePartitions); } - // If related base table create partitions or mv is created with ttl, need base table union + // If related base table creates partitions or mv is created with ttl, need base table union Sets.difference(queryUsedBaseTablePartitionNameSet, mvValidBaseTablePartitionNameSet) .copyInto(baseTableNeedUnionPartitionNameSet); // Construct result map - Map> mvPartitionNeedRemoveNameMap = new HashMap<>(); + Pair> mvPartitionNeedRemoveNameMap = Pair.of( + new BaseTableInfo(mtmv), ImmutableSet.of()); if (!mvNeedRemovePartitionNameSet.isEmpty()) { - mvPartitionNeedRemoveNameMap.put(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet); + mvPartitionNeedRemoveNameMap = Pair.of(new BaseTableInfo(mtmv), mvNeedRemovePartitionNameSet); } - Map> baseTablePartitionNeedUnionNameMap = new HashMap<>(); + Pair> baseTablePartitionNeedUnionNameMap = Pair.of( + relatedPartitionTable, ImmutableSet.of()); if (!baseTableNeedUnionPartitionNameSet.isEmpty()) { - baseTablePartitionNeedUnionNameMap.put(relatedPartitionTable, baseTableNeedUnionPartitionNameSet); + baseTablePartitionNeedUnionNameMap = Pair.of(relatedPartitionTable, baseTableNeedUnionPartitionNameSet); } return Pair.of(mvPartitionNeedRemoveNameMap, baseTablePartitionNeedUnionNameMap); } public static boolean needUnionRewrite( - Pair>, Map>> invalidPartitions, + Pair>, Map>> invalidPartitions, CascadesContext cascadesContext) { return invalidPartitions != null - && (!invalidPartitions.key().isEmpty() || !invalidPartitions.value().isEmpty()); + && (!invalidPartitions.key().values().isEmpty() || !invalidPartitions.value().values().isEmpty()); } /** @@ -158,8 +230,8 @@ public static boolean needUnionRewrite(MaterializationContext materializationCon } MTMV mtmv = ((AsyncMaterializationContext) materializationContext).getMtmv(); PartitionType type = mtmv.getPartitionInfo().getType(); - BaseTableInfo relatedTableInfo = mtmv.getMvPartitionInfo().getRelatedTableInfo(); - return !PartitionType.UNPARTITIONED.equals(type) && relatedTableInfo != null; + List pctInfos = mtmv.getMvPartitionInfo().getPctInfos(); + return !PartitionType.UNPARTITIONED.equals(type) && !pctInfos.isEmpty(); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionIncrementMaintainer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionIncrementMaintainer.java new file mode 100644 index 00000000000000..7325bc0217b0b7 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/PartitionIncrementMaintainer.java @@ -0,0 +1,845 @@ +// 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.Column; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.mvcc.MvccUtil; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.mtmv.MTMVRelatedTableIf; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo.RelatedTableColumnInfo; +import org.apache.doris.nereids.rules.expression.ExpressionNormalization; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.WindowExpression; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; +import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.SetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +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.LogicalLimit; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.BitSet; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +/** + * Partition Increment Maintainer, this is used to check whether the materialized view can be partition level + * increment maintained + */ +public class PartitionIncrementMaintainer { + + /** + * Partition Increment Checker + */ + public static class PartitionIncrementChecker extends + DefaultPlanVisitor { + public static final PartitionIncrementChecker INSTANCE = new PartitionIncrementChecker(); + public static final Set> SUPPORT_EXPRESSION_TYPES = + ImmutableSet.of(DateTrunc.class, SlotReference.class, Literal.class); + + @Override + public Void visitLogicalProject(LogicalProject project, + PartitionIncrementCheckContext context) { + List output = project.getOutput(); + boolean isValid = checkPartition(output, project, context); + if (!isValid) { + context.collectFailedTableSet(project); + } + return visit(project, context); + } + + @Override + public Void visitLogicalFilter(LogicalFilter filter, PartitionIncrementCheckContext context) { + return visit(filter, context); + } + + @Override + public Void visitLogicalUnion(LogicalUnion union, PartitionIncrementCheckContext context) { + Set checkingNamedExpressionSet = new HashSet<>( + context.getPartitionAndRefExpressionMap().keySet()); + int index = -1; + List output = union.getOutput(); + for (int j = 0; j < output.size(); j++) { + if (checkingNamedExpressionSet.contains(output.get(j))) { + index = j; + break; + } + } + if (index == -1) { + context.addFailReason("union all output doesn't contain the target partition"); + context.collectFailedTableSet(union); + return null; + } + List children = union.children(); + List childrenContextList = new ArrayList<>(); + Slot unionSlotToCheck = output.get(index); + for (int i = 0; i < children.size(); i++) { + List regularChildOutput = union.getRegularChildOutput(i); + SlotReference childMvPartitionSlot = regularChildOutput.get(index); + Optional childPartitionExpression = replace(childMvPartitionSlot, unionSlotToCheck, + context.getPartitionAndRefExpressionMap().get(unionSlotToCheck) + .getPartitionExpression()); + PartitionIncrementCheckContext childContext = new PartitionIncrementCheckContext(childMvPartitionSlot, + childPartitionExpression.orElse(null), context.getProducerCteIdToPlanMap(), + children.get(i), context.getCascadesContext()); + children.get(i).accept(this, childContext); + childrenContextList.add(childContext); + } + boolean allReachRelationCheck = true; + boolean allIsFromTablePartitionColumn = true; + for (PartitionIncrementCheckContext childContext : childrenContextList) { + boolean childAnyIsFromTablePartitionColumn = false; + boolean childAnyReachRelationCheck = false; + for (RelatedTableColumnInfo tableColumnInfo : childContext.getPartitionAndRefExpressionMap().values()) { + if (tableColumnInfo.isReachRelationCheck()) { + childAnyReachRelationCheck = true; + } + childAnyIsFromTablePartitionColumn + = childAnyIsFromTablePartitionColumn || tableColumnInfo.isFromTablePartitionColumn(); + } + if (!childAnyReachRelationCheck) { + context.addFailReason(String.format( + "union all output doesn't match the partition increment check, fail reason is %s", + childContext.getFailReasons())); + allReachRelationCheck = false; + break; + } + allIsFromTablePartitionColumn = allIsFromTablePartitionColumn && childAnyIsFromTablePartitionColumn; + } + if (allReachRelationCheck && allIsFromTablePartitionColumn) { + childrenContextList.forEach( + childContext -> context.getPartitionAndRefExpressionMap().putAll( + childContext.getPartitionAndRefExpressionMap())); + } else { + context.collectFailedTableSet(union); + context.addFailReason("not union all output pass partition increment check"); + } + return super.visit(union, context); + } + + @Override + public Void visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, PartitionIncrementCheckContext context) { + Plan producerPlan = context.getProducerCteIdToPlanMap().get(cteConsumer.getCteId()); + if (producerPlan == null) { + context.addFailReason(String.format("can't find cte producer producerPlan by cte id %s", + cteConsumer.getCteId())); + return null; + } + Map consumerToProducerOutputMap = cteConsumer.getConsumerToProducerOutputMap(); + Map needAddMap = new HashMap<>(); + for (Map.Entry entry + : context.getPartitionAndRefExpressionMap().entrySet()) { + NamedExpression consumerSlot = entry.getKey(); + Slot producerSlot = consumerToProducerOutputMap.get(consumerSlot); + if (producerSlot == null) { + continue; + } + needAddMap.put(producerSlot, RelatedTableColumnInfo.of(producerSlot, + replace(producerSlot, consumerSlot, entry.getValue().getPartitionExpression()).orElse(null), + entry.getValue().isOriginalPartition(), + entry.getValue().isFromTablePartitionColumn())); + // clac the equal set in context + Set> shuttledEqualSlotSet = context.getShuttledEqualSlotSet(); + for (Set equalSlotSet : shuttledEqualSlotSet) { + if (equalSlotSet.contains(consumerSlot)) { + Expression shuttledSlot = ExpressionUtils.shuttleExpressionWithLineage( + producerSlot, producerPlan, new BitSet()); + if (shuttledSlot instanceof Slot) { + equalSlotSet.add((Slot) shuttledSlot); + } + } + } + } + if (!needAddMap.isEmpty()) { + context.getPartitionAndRefExpressionMap().putAll(needAddMap); + } + return super.visit(producerPlan, context); + } + + @Override + public Void visitLogicalCTEProducer(LogicalCTEProducer cteProducer, + PartitionIncrementCheckContext context) { + // should visit by logical cte consumer + return null; + } + + @Override + public Void visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + PartitionIncrementCheckContext context) { + return super.visitLogicalCTEAnchor(cteAnchor, context); + } + + @Override + public Void visitLogicalJoin(LogicalJoin join, + PartitionIncrementCheckContext context) { + if (join.isMarkJoin()) { + context.addFailReason("partition track doesn't support mark join"); + return null; + } + // calculate equal slot set from join condition + Map toAdd = new HashMap<>(); + for (NamedExpression partitionSlotToCheck : context.getPartitionAndRefExpressionMap().keySet()) { + if (!(partitionSlotToCheck instanceof SlotReference)) { + continue; + } + Pair, Set> partitionEqualSlotPair = + calEqualSet((SlotReference) partitionSlotToCheck, join); + if (!partitionEqualSlotPair.value().isEmpty()) { + context.getShuttledEqualSlotSet().add(partitionEqualSlotPair.value()); + } + for (Slot partitionEqualSlot : partitionEqualSlotPair.key()) { + // If equal slot set founded, add the slot and ref expression to checker context + Optional replacedPartitionExpression = replace(partitionEqualSlot, partitionSlotToCheck, + context.getPartitionAndRefExpressionMap().get(partitionSlotToCheck) + .getPartitionExpression()); + toAdd.put(partitionEqualSlot, + RelatedTableColumnInfo.of(partitionEqualSlot, replacedPartitionExpression.orElse(null), + false, false)); + } + } + if (!toAdd.isEmpty()) { + context.getPartitionAndRefExpressionMap().putAll(toAdd); + } + // check join type and partition column side + Set leftColumnSet = join.child(0).getOutputSet(); + Set namedExpressions = new HashSet<>(context.getPartitionAndRefExpressionMap().keySet()); + for (NamedExpression partitionSlotToCheck : namedExpressions) { + if (!(partitionSlotToCheck instanceof SlotReference)) { + continue; + } + boolean useLeft = leftColumnSet.contains(partitionSlotToCheck); + JoinType joinType = join.getJoinType(); + if (joinType.isInnerJoin() || joinType.isCrossJoin()) { + visit(join, context); + } else if ((joinType.isLeftJoin() + || joinType.isLeftSemiJoin() + || joinType.isLeftAntiJoin()) && useLeft) { + context.collectInvalidTableSet(join.right()); + visit(join, context); + } else if ((joinType.isRightJoin() + || joinType.isRightAntiJoin() + || joinType.isRightSemiJoin()) && !useLeft) { + context.collectInvalidTableSet(join.left()); + visit(join, context); + } else { + context.addFailReason(String.format("partition column is in un supported join null generate side, " + + "current join type is %s, partitionSlot is %s", joinType, partitionSlotToCheck)); + } + } + return null; + } + + @Override + public Void visitLogicalRelation(LogicalRelation relation, PartitionIncrementCheckContext context) { + if (!(relation instanceof LogicalCatalogRelation)) { + context.addFailReason(String.format("relation should be LogicalCatalogRelation, " + + "but now is %s", relation.getClass().getSimpleName())); + return null; + } + LogicalCatalogRelation logicalCatalogRelation = (LogicalCatalogRelation) relation; + TableIf table = logicalCatalogRelation.getTable(); + if (!(table instanceof MTMVRelatedTableIf)) { + context.addFailReason(String.format("relation base table is not MTMVRelatedTableIf, the table is %s", + table.getName())); + return null; + } + List relatedTableColumnInfosByTable = getRelatedTableColumnInfosByTable(context, + new BaseTableInfo(table)); + // mark reach relation check + if (!context.getInvalidCatalogRelationToCheck().contains(relation) + && !context.getShouldFailCatalogRelation().contains(relation)) { + relatedTableColumnInfosByTable.forEach( + tableColumnInfo -> tableColumnInfo.setReachRelationCheck(true)); + } + MTMVRelatedTableIf relatedTable = (MTMVRelatedTableIf) table; + PartitionType type = relatedTable.getPartitionType(MvccUtil.getSnapshotFromContext(relatedTable)); + if (PartitionType.UNPARTITIONED.equals(type)) { + context.addFailReason(String.format("related base table is not partition table, the table is %s", + table.getName())); + return null; + } + Set relatedTablePartitionColumnSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); + relatedTablePartitionColumnSet.addAll(relatedTable.getPartitionColumns( + MvccUtil.getSnapshotFromContext(relatedTable)).stream() + .map(Column::getName).collect(Collectors.toSet())); + for (Map.Entry contextPartitionColumnEntry + : context.getPartitionAndRefExpressionMap().entrySet()) { + NamedExpression partitionNamedExpression = contextPartitionColumnEntry.getKey(); + if (!(partitionNamedExpression instanceof SlotReference)) { + continue; + } + SlotReference contextPartitionColumn = (SlotReference) partitionNamedExpression; + if (!contextPartitionColumn.getOriginalTable() + .map(TableIf::getFullQualifiers).orElse(ImmutableList.of()) + .equals(((LogicalCatalogRelation) relation).getTable().getFullQualifiers())) { + // mv partition column name is not belonged to current table, continue check + continue; + } + RelatedTableColumnInfo mvReferenceColumnInfo = contextPartitionColumnEntry.getValue(); + Column mvReferenceColumnToCheck = mvReferenceColumnInfo.getColumn(); + if (mvReferenceColumnToCheck == null) { + context.addFailReason(String.format("related base table mvReferenceColumnToCheck is null" + + " mvReferenceColumnInfo is %s", mvReferenceColumnInfo)); + continue; + } + if (!relatedTablePartitionColumnSet.contains(mvReferenceColumnToCheck.getName())) { + context.addFailReason(String.format("related base table partition column doesn't contain the mv" + + " partition, the mvReferenceColumnToCheck is %s", + mvReferenceColumnToCheck)); + continue; + } + if ((mvReferenceColumnToCheck.isAllowNull() && !relatedTable.isPartitionColumnAllowNull())) { + context.addFailReason(String.format("related base table partition column" + + " partition nullable check fail, the mvReferenceColumnToCheck is %s", + mvReferenceColumnToCheck)); + continue; + } + SlotReference currentPartitionSlot = null; + for (Slot catalogSlot : logicalCatalogRelation.getOutputSet()) { + if (catalogSlot instanceof SlotReference + && Objects.equals(((SlotReference) catalogSlot).getOriginalColumn().orElse(null), + mvReferenceColumnToCheck)) { + currentPartitionSlot = (SlotReference) catalogSlot; + break; + } + } + // If self join such as inner join + // should also check the partition column is in the shuttled equal set + boolean tableHasChecked = context.getPartitionAndRefExpressionMap().values().stream() + .anyMatch(tableColumnInfo -> + tableColumnInfo.isFromTablePartitionColumn() + && new BaseTableInfo(((LogicalCatalogRelation) relation).getTable()) + .equals(tableColumnInfo.getTableInfo()) + ); + if (tableHasChecked || context.getInvalidCatalogRelationToCheck().contains(relation)) { + boolean checkSuccess = false; + for (Set equalSlotSet : context.getShuttledEqualSlotSet()) { + checkSuccess = equalSlotSet.contains(contextPartitionColumn) + && equalSlotSet.contains(currentPartitionSlot); + if (checkSuccess) { + break; + } + } + if (!checkSuccess) { + context.addFailReason(String.format("partition column is in join invalid side, " + + "but is not in join condition, the mvReferenceColumnToCheck is %s", + mvReferenceColumnToCheck)); + relatedTableColumnInfosByTable.forEach( + columnInfo -> columnInfo.setFromTablePartitionColumn(false)); + } else { + mvReferenceColumnInfo.setReachRelationCheck(true); + mvReferenceColumnInfo.setFromTablePartitionColumn(true); + } + } + if (context.getShouldFailCatalogRelation().stream().noneMatch(catalog -> + new BaseTableInfo(catalog.getTable()).equals(mvReferenceColumnInfo.getTableInfo())) + && context.getInvalidCatalogRelationToCheck().stream().noneMatch(catalog -> + new BaseTableInfo(catalog.getTable()).equals(mvReferenceColumnInfo.getTableInfo()))) { + mvReferenceColumnInfo.setReachRelationCheck(true); + mvReferenceColumnInfo.setFromTablePartitionColumn(true); + } else { + context.addFailReason(String.format("partition column is in invalid catalog relation to check, " + + "InvalidCatalogRelationToCheck is %s, ShouldFailCatalogRelation is %s", + context.getInvalidCatalogRelationToCheck().stream() + .map(LogicalCatalogRelation::getTable) + .collect(Collectors.toList()), + context.getShouldFailCatalogRelation().stream() + .map(LogicalCatalogRelation::getTable) + .collect(Collectors.toList()))); + } + } + return visit(relation, context); + } + + @Override + public Void visitLogicalAggregate(LogicalAggregate aggregate, + PartitionIncrementCheckContext context) { + Set groupByExprSet = new HashSet<>(aggregate.getGroupByExpressions()); + if (groupByExprSet.isEmpty()) { + context.addFailReason("group by sets is empty, doesn't contain the target partition"); + context.setFailFast(true); + context.collectFailedTableSet(aggregate); + return visit(aggregate, context); + } + if (!checkPartition(groupByExprSet, aggregate, context)) { + context.setFailFast(true); + context.collectFailedTableSet(aggregate); + } + return visit(aggregate, context); + } + + @Override + public Void visitLogicalWindow(LogicalWindow window, PartitionIncrementCheckContext context) { + List windowExpressions = window.getWindowExpressions(); + if (windowExpressions.isEmpty()) { + context.addFailReason("window expression is empty, doesn't contain the target partition"); + context.collectFailedTableSet(window); + context.setFailFast(true); + return visit(window, context); + } + for (NamedExpression namedExpression : windowExpressions) { + if (!checkWindowPartition(namedExpression, context)) { + context.addFailReason("window partition sets doesn't contain the target partition"); + context.collectFailedTableSet(window); + context.setFailFast(true); + break; + } + } + return super.visitLogicalWindow(window, context); + } + + @Override + public Void visit(Plan plan, PartitionIncrementCheckContext context) { + if (plan instanceof LogicalProject + || plan instanceof LogicalLimit + || plan instanceof LogicalFilter + || plan instanceof LogicalJoin + || plan instanceof LogicalAggregate + || plan instanceof LogicalCatalogRelation + || plan instanceof LogicalResultSink + || plan instanceof LogicalWindow + || (plan instanceof LogicalUnion + && ((LogicalUnion) plan).getQualifier() == SetOperation.Qualifier.ALL) + || plan instanceof LogicalCTEAnchor + || plan instanceof LogicalCTEConsumer + || plan instanceof LogicalCTEProducer) { + return super.visit(plan, context); + } + context.addFailReason(String.format("Unsupported plan operate in track partition, " + + "the invalid plan node is %s", plan.getClass().getSimpleName())); + context.collectFailedTableSet(plan); + return super.visit(plan, context); + } + + private boolean checkWindowPartition(Expression expression, PartitionIncrementCheckContext context) { + List windowExpressions = + expression.collectToList(expressionTreeNode -> expressionTreeNode instanceof WindowExpression); + for (Object windowExpressionObj : windowExpressions) { + WindowExpression windowExpression = (WindowExpression) windowExpressionObj; + List partitionKeys = windowExpression.getPartitionKeys(); + Set originalPartitionbyExprSet = new HashSet<>(); + partitionKeys.forEach(groupExpr -> { + if (groupExpr instanceof SlotReference && groupExpr.isColumnFromTable()) { + originalPartitionbyExprSet.add(((SlotReference) groupExpr).getOriginalColumn().get()); + } + }); + Set contextPartitionColumnSet = getPartitionColumnsToCheck(context); + if (contextPartitionColumnSet.isEmpty()) { + return false; + } + if (contextPartitionColumnSet.stream().noneMatch( + partition -> originalPartitionbyExprSet.contains(partition.getOriginalColumn().get()))) { + return false; + } + } + return true; + } + + private Set getPartitionColumnsToCheck(PartitionIncrementCheckContext context) { + Set partitionExpressionSet = context.getPartitionAndRefExpressionMap().keySet(); + Set partitionSlotSet = new HashSet<>(); + for (NamedExpression namedExpression : partitionExpressionSet) { + if (!namedExpression.isColumnFromTable()) { + context.addFailReason(String.format("context partition column should be slot from column, " + + "context column is %s", namedExpression)); + continue; + } + partitionSlotSet.add((SlotReference) namedExpression); + } + return partitionSlotSet; + } + + /** + * Given a partition named expression and expressionsToCheck, check the partition is valid + * example 1: + * partition expression is date_trunc(date_alias#25, 'hour') AS `date_trunc(date_alias, 'hour')`#30 + * expressionsToCheck is date_trunc(date_alias, 'hour')#30 + * expressionsToCheck is the slot to partition expression, but they are expression + * example 2: + * partition expression is L_SHIPDATE#10 + * expressionsToCheck isL_SHIPDATE#10 + * both of them are slot + * example 3: + * partition expression is date_trunc(L_SHIPDATE#10, 'hour')#30 + * expressionsToCheck is L_SHIPDATE#10 + * all above should check successfully + */ + private static boolean checkPartition(Collection expressionsToCheck, Plan plan, + PartitionIncrementCheckContext context) { + Set> partitionAndExprEntrySet + = new HashSet<>(context.getPartitionAndRefExpressionMap().entrySet()); + boolean checked = false; + for (Map.Entry partitionExpressionEntry + : partitionAndExprEntrySet) { + NamedExpression partitionNamedExpression = partitionExpressionEntry.getKey(); + RelatedTableColumnInfo partitionTableColumnInfo = partitionExpressionEntry.getValue(); + Optional partitionExpressionOpt = partitionTableColumnInfo.getPartitionExpression(); + Expression partitionExpressionActual = partitionExpressionOpt.orElseGet( + () -> ExpressionUtils.shuttleExpressionWithLineage(partitionNamedExpression, + context.getOriginalPlan(), new BitSet())); + // merge date_trunc + partitionExpressionActual = new ExpressionNormalization().rewrite(partitionExpressionActual, + new ExpressionRewriteContext(context.getCascadesContext())); + OUTER_CHECK: + for (Expression projectSlotToCheck : expressionsToCheck) { + Expression expressionShuttledToCheck = + ExpressionUtils.shuttleExpressionWithLineage(projectSlotToCheck, + context.getOriginalPlan(), new BitSet()); + // merge date_trunc + expressionShuttledToCheck = new ExpressionNormalization().rewrite(expressionShuttledToCheck, + new ExpressionRewriteContext(context.getCascadesContext())); + + Set expressionToCheckSlots = + expressionShuttledToCheck.collectToSet(SlotReference.class::isInstance); + Set partitionColumnSlots = + partitionExpressionActual.collectToSet(SlotReference.class::isInstance); + if (Sets.intersection(expressionToCheckSlots, partitionColumnSlots).isEmpty() + || expressionToCheckSlots.isEmpty() || partitionColumnSlots.isEmpty()) { + // this expression doesn't use partition column + continue; + } + if (expressionToCheckSlots.size() != 1 || partitionColumnSlots.size() != 1) { + context.addFailReason( + String.format("partition expression use more than one slot reference, invalid " + + "expressionToCheckSlots is %s, partitionColumnDateColumns is %s", + expressionToCheckSlots, partitionColumnSlots)); + continue; + } + List expressionsToCheckList = expressionShuttledToCheck.collectToList( + Expression.class::isInstance); + for (Expression expression : expressionsToCheckList) { + if (SUPPORT_EXPRESSION_TYPES.stream().noneMatch( + supportExpression -> supportExpression.isAssignableFrom(expression.getClass()))) { + context.addFailReason( + String.format("column to check use invalid implicit expression, invalid " + + "expression is %s", expression)); + continue OUTER_CHECK; + } + } + List partitionExpressionList = partitionExpressionActual.collectToList( + Expression.class::isInstance); + for (Expression expression : partitionExpressionList) { + if (SUPPORT_EXPRESSION_TYPES.stream().noneMatch( + supportExpression -> supportExpression.isAssignableFrom(expression.getClass()))) { + context.addFailReason( + String.format("partition column use invalid implicit expression, invalid " + + "expression is %s", expression)); + continue OUTER_CHECK; + } + } + List expressionToCheckDataTruncList = + expressionShuttledToCheck.collectToList(DateTrunc.class::isInstance); + List partitionExpressionDateTrucList = + partitionExpressionActual.collectToList(DateTrunc.class::isInstance); + if (expressionToCheckDataTruncList.size() > 1 || partitionExpressionDateTrucList.size() > 1) { + // mv time unit level is little then query + context.addFailReason("partition column time unit level should be " + + "greater than sql select column"); + continue; + } + SlotReference checkedPartitionSlot = partitionColumnSlots.iterator().next(); + context.getPartitionAndRefExpressionMap().put(checkedPartitionSlot, + RelatedTableColumnInfo.of(checkedPartitionSlot, partitionExpressionActual, + partitionTableColumnInfo.isOriginalPartition(), + partitionTableColumnInfo.isFromTablePartitionColumn())); + checked = true; + } + } + return checked; + } + } + + /** + * Get valid related table column info from check context by predicate + */ + public static List getRelatedTableColumnInfosWithCheck( + PartitionIncrementCheckContext checkContext, + Predicate predicate) { + Set dataTypeSet = new HashSet<>(); + List checkedTableColumnInfos = new ArrayList<>(); + boolean anyIsFromTablePartitionColumn = false; + // if predicate use isReachRelationCheck, this also need to check isFromTablePartitionColumn + Set checkedTableSet = new HashSet<>(); + for (Map.Entry entry + : checkContext.getPartitionAndRefExpressionMap().entrySet()) { + NamedExpression partitionColumn = entry.getKey(); + RelatedTableColumnInfo tableColumnInfo = entry.getValue(); + if (!partitionColumn.isColumnFromTable() || tableColumnInfo.getColumn() == null) { + continue; + } + dataTypeSet.add(partitionColumn.getDataType()); + if (dataTypeSet.size() > 1) { + return null; + } + if (checkedTableSet.contains(tableColumnInfo.getTableInfo())) { + // remove duplicate table info + continue; + } + if (predicate.test(tableColumnInfo)) { + checkedTableColumnInfos.add(tableColumnInfo); + checkedTableSet.add(tableColumnInfo.getTableInfo()); + } + anyIsFromTablePartitionColumn + = anyIsFromTablePartitionColumn || tableColumnInfo.isFromTablePartitionColumn(); + } + return anyIsFromTablePartitionColumn ? checkedTableColumnInfos : ImmutableList.of(); + } + + /** + * Get valid related table column info from check context by predicate + */ + public static List getRelatedTableColumnInfosByTable( + PartitionIncrementCheckContext checkContext, + BaseTableInfo tableInfo) { + List checkedTableColumnInfos = new ArrayList<>(); + for (Map.Entry entry + : checkContext.getPartitionAndRefExpressionMap().entrySet()) { + RelatedTableColumnInfo tableColumnInfo = entry.getValue(); + if (tableColumnInfo.getTableInfo() != null && tableColumnInfo.getTableInfo().equals(tableInfo)) { + checkedTableColumnInfos.add(tableColumnInfo); + } + } + return checkedTableColumnInfos; + } + + /** + * The context used in IncrementChecker + */ + public static final class PartitionIncrementCheckContext { + // This is used to record partition slot, and the value of map is ref date expression and bool value which + // identify it's original partition or not, the key of map is the namedExpression to check + private final Map partitionAndRefExpressionMap + = new HashMap<>(); + private final Set failReasons = new HashSet<>(); + private final CascadesContext cascadesContext; + // This record the invalid relation, such as the right side of left join + private final Set invalidCatalogRelationToCheck = new HashSet<>(); + // This record should fail relation, such as invalid child of checked plan node + private final Set shouldFailCatalogRelation = new HashSet<>(); + // This is used to record the equal slot set shuttled from children which are equals to partition column + // to check, this expends the partition slot to check + private final Set> shuttledEqualSlotSet = new HashSet<>(); + private final Map producerCteIdToPlanMap; + private final Plan originalPlan; + private boolean failFast = false; + + public PartitionIncrementCheckContext(NamedExpression mvPartitionColumn, + Expression mvPartitionExpression, Map producerCteIdToPlanMap, + Plan originalPlan, + CascadesContext cascadesContext) { + this.partitionAndRefExpressionMap.put(mvPartitionColumn, RelatedTableColumnInfo.of( + mvPartitionColumn, mvPartitionExpression, true, false)); + this.cascadesContext = cascadesContext; + this.producerCteIdToPlanMap = producerCteIdToPlanMap; + this.originalPlan = originalPlan; + } + + public Set getFailReasons() { + return failReasons; + } + + public void addFailReason(String failReason) { + this.failReasons.add(failReason); + } + + public Set getInvalidCatalogRelationToCheck() { + return invalidCatalogRelationToCheck; + } + + public Set getShouldFailCatalogRelation() { + return shouldFailCatalogRelation; + } + + public CascadesContext getCascadesContext() { + return cascadesContext; + } + + public Set> getShuttledEqualSlotSet() { + return shuttledEqualSlotSet; + } + + public Map getPartitionAndRefExpressionMap() { + return partitionAndRefExpressionMap; + } + + public Map getProducerCteIdToPlanMap() { + return producerCteIdToPlanMap; + } + + public boolean isFailFast() { + return failFast; + } + + public void setFailFast(boolean failFast) { + this.failFast = failFast; + } + + public Plan getOriginalPlan() { + return originalPlan; + } + + /** + * collect invalid table set to check self join + */ + public void collectInvalidTableSet(Plan plan) { + plan.accept(new DefaultPlanVisitor>() { + @Override + public Void visitLogicalCatalogRelation(LogicalCatalogRelation relation, + Set invalidTableSet) { + invalidTableSet.add(relation); + return null; + } + }, this.invalidCatalogRelationToCheck); + } + + /** + * collect invalid table set to check self join + */ + public void collectFailedTableSet(Plan plan) { + plan.accept(new DefaultPlanVisitor>() { + @Override + public Void visitLogicalCatalogRelation(LogicalCatalogRelation relation, + Set failedTableSet) { + failedTableSet.add(relation); + return null; + } + }, this.shouldFailCatalogRelation); + } + } + + /** + * Add partitionEqualSlot to partitionAndRefExpressionToCheck if partitionExpression use the partitionSlot + */ + private static Optional replace(NamedExpression partitionEqualSlot, + NamedExpression partitionSlot, + Optional partitionExpression) { + if (Objects.equals(partitionSlot, partitionEqualSlot)) { + return partitionExpression; + } + if (!partitionExpression.isPresent()) { + return Optional.empty(); + } + // Replace partitionSlot in partitionExpression with partitionEqualSlot + Expression replacedExpression = partitionExpression.map( + partitionExpr -> partitionExpr.accept(new DefaultExpressionRewriter() { + @Override + public Expression visitNamedExpression(NamedExpression namedExpression, Void context) { + if (namedExpression.equals(partitionSlot)) { + return partitionEqualSlot; + } + return namedExpression; + } + }, null)).orElse(null); + // if replacedExpression doesn't contain partitionSlot which means replace successfully, then add + if (replacedExpression == null) { + return Optional.empty(); + } + Set partitionSlotSet = + replacedExpression.collectToSet(expr -> expr.equals(partitionSlot)); + if (partitionSlotSet.isEmpty()) { + // If replaced successfully, then add partitionEqualSlot to partition and ref + // expression map to check + return Optional.of(replacedExpression); + } + return Optional.empty(); + } + + /** + * the key of result is the equal slot set to slot, which are not shuttled + * the value of result is the equal slot set to slot, which are shuttled from children + * the key equal set should not contain the slot itself + * the value equal set contain the slot itself + */ + private static Pair, Set> calEqualSet(Slot slot, + LogicalJoin join) { + Set partitionEqualSlotSet = new HashSet<>(); + JoinType joinType = join.getJoinType(); + if (joinType.isInnerJoin() || joinType.isSemiJoin()) { + partitionEqualSlotSet.addAll(join.getLogicalProperties().getTrait().calEqualSet(slot)); + } + // construct shuttled partitionEqualSlotSet + Set shuttledPartitionEqualSlotSet = new HashSet<>(); + if (partitionEqualSlotSet.isEmpty()) { + return Pair.of(partitionEqualSlotSet, shuttledPartitionEqualSlotSet); + } + List extendedPartitionEqualSlotSet = new ArrayList<>(partitionEqualSlotSet); + extendedPartitionEqualSlotSet.add(slot); + List shuttledEqualExpressions = ExpressionUtils.shuttleExpressionWithLineage( + extendedPartitionEqualSlotSet, join, new BitSet()); + for (Expression shuttledEqualExpression : shuttledEqualExpressions) { + Set objects = shuttledEqualExpression.collectToSet(expr -> expr instanceof SlotReference); + if (objects.size() != 1 || !(shuttledEqualExpression instanceof SlotReference)) { + continue; + } + shuttledPartitionEqualSlotSet.add((Slot) shuttledEqualExpression); + } + return Pair.of(partitionEqualSlotSet, shuttledPartitionEqualSlotSet); + } + + /** + * Remove the sink node from materialized view plan + */ + public static Plan removeSink(Plan materializedViewPlan) { + return materializedViewPlan.accept(new DefaultPlanRewriter() { + @Override + public Plan visitLogicalSink(LogicalSink logicalSink, Void context) { + return new LogicalProject<>(logicalSink.getOutputExprs(), logicalSink.child()); + } + }, null); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelatedTableInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelatedTableInfo.java new file mode 100644 index 00000000000000..fe3accbbf6140b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelatedTableInfo.java @@ -0,0 +1,214 @@ +// 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.analysis.Expr; +import org.apache.doris.analysis.SlotRef; +import org.apache.doris.catalog.Column; +import org.apache.doris.mtmv.BaseTableInfo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +/** + * The related partition table info which mv relate + */ +public class RelatedTableInfo { + private final boolean pctPossible; + private final Set failReasons = new HashSet<>(); + private final List tableColumnInfos; + + public RelatedTableInfo(boolean pctPossible, List tableColumnInfos, + Set failReasons) { + this.pctPossible = pctPossible; + this.failReasons.addAll(failReasons); + this.tableColumnInfos = tableColumnInfos; + } + + public static RelatedTableInfo successWith(List tableColumnInfos) { + return new RelatedTableInfo(true, tableColumnInfos, ImmutableSet.of()); + } + + public static RelatedTableInfo failWith(String failReason) { + return new RelatedTableInfo(false, ImmutableList.of(), ImmutableSet.of(failReason)); + } + + public static RelatedTableInfo failWith(Set failReasons) { + return new RelatedTableInfo(false, ImmutableList.of(), failReasons); + } + + public boolean isPctPossible() { + return pctPossible; + } + + public String getFailReason() { + return String.join(",", failReasons); + } + + public Set getFailReasons() { + return failReasons; + } + + /** + * Get valid table column infos + */ + public List getTableColumnInfos() { + List validTableColumnInfos = new ArrayList<>(); + for (RelatedTableColumnInfo relatedTableColumnInfo : tableColumnInfos) { + if (relatedTableColumnInfo.getColumn() != null) { + validTableColumnInfos.add(relatedTableColumnInfo); + } + } + return validTableColumnInfos; + } + + /** + * The related table and partition column info which mv relates + */ + public static final class RelatedTableColumnInfo { + // This records the partition named expression + private final NamedExpression partitionNamedExpression; + // This records the partition rollup expression if exist + private final Optional partitionExpression; + // This record the partition column is original or derived from equal set + private final boolean isOriginalPartition; + // This partition column to check is in the table's partition columns, if not this is false + // if in the table's partition columns, this is true, this is used to derive union all partition column + private boolean isFromTablePartitionColumn; + private boolean isReachRelationCheck; + + private RelatedTableColumnInfo(NamedExpression partitionNamedExpression, + Expression partitionExpression, + boolean isOriginalPartition, + boolean isFromTablePartitionColumn) { + this.partitionNamedExpression = partitionNamedExpression; + this.partitionExpression = Optional.ofNullable(partitionExpression); + this.isOriginalPartition = isOriginalPartition; + this.isFromTablePartitionColumn = isFromTablePartitionColumn; + } + + /** + * get partition table info + */ + public BaseTableInfo getTableInfo() { + if (!(partitionNamedExpression instanceof SlotReference) + || !partitionNamedExpression.isColumnFromTable()) { + return null; + } + return new BaseTableInfo(((SlotReference) partitionNamedExpression).getOriginalTable().get()); + } + + /** + * get column str + */ + public String getColumnStr() { + Column column = getColumn(); + return column == null ? null : column.getName(); + } + + /** + * get column + */ + public Column getColumn() { + if (!(partitionNamedExpression instanceof SlotReference) + || !partitionNamedExpression.isColumnFromTable()) { + return null; + } + return extractColumn(this.partitionNamedExpression); + } + + public NamedExpression getPartitionNamedExpression() { + return partitionNamedExpression; + } + + public Optional getPartitionExpression() { + return partitionExpression; + } + + public boolean isOriginalPartition() { + return isOriginalPartition; + } + + public boolean isFromTablePartitionColumn() { + return isFromTablePartitionColumn; + } + + public void setFromTablePartitionColumn(boolean fromTablePartitionColumn) { + isFromTablePartitionColumn = fromTablePartitionColumn; + } + + public boolean isReachRelationCheck() { + return isReachRelationCheck; + } + + public void setReachRelationCheck(boolean reachRelationCheck) { + isReachRelationCheck = reachRelationCheck; + } + + public static RelatedTableColumnInfo of(NamedExpression partitionNamedExpression, + Expression partitionExpression, boolean isOriginalPartition, boolean isFromTablePartitionColumn) { + return new RelatedTableColumnInfo(partitionNamedExpression, partitionExpression, + isOriginalPartition, isFromTablePartitionColumn); + } + + /** + * Extract column from slot reference + */ + private static Column extractColumn(NamedExpression slotReference) { + if (!(slotReference instanceof SlotReference)) { + return null; + } + Optional slotReferenceColumn = ((SlotReference) slotReference).getOriginalColumn(); + if (!slotReferenceColumn.isPresent()) { + return null; + } + if (!slotReference.isColumnFromTable()) { + // Column is not from table + return null; + } + Expr definExpr = slotReferenceColumn.get().getDefineExpr(); + if (definExpr instanceof SlotRef) { + // If slotReference is from sync mv when rbo, should get actual column + Column referenceRollupColumn = ((SlotRef) definExpr).getColumn(); + if (referenceRollupColumn != null) { + return referenceRollupColumn; + } + } + return slotReferenceColumn.get(); + } + + @Override + public String toString() { + return "RelatedTableColumnInfo{" + + "partitionNamedExpression=" + partitionNamedExpression + + ", partitionExpression=" + partitionExpression + + ", isOriginalPartition=" + isOriginalPartition + + ", isFromTablePartitionColumn=" + isFromTablePartitionColumn + + ", isReachRelationCheck=" + isReachRelationCheck + '}'; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java index fedd2ece30b783..89964b0c8d20fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.rules.exploration.mv; import org.apache.doris.common.Pair; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.executor.Rewriter; @@ -725,12 +726,11 @@ public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, * need to add filter. * return null if add filter fail. */ - public static Pair addFilterOnTableScan(Plan queryPlan, Map> partitionOnOriginPlan, String partitionColumn, CascadesContext parentCascadesContext) { + public static Pair addFilterOnTableScan(Plan queryPlan, + Map> partitionOnBaseTableMap, CascadesContext parentCascadesContext) { // Firstly, construct filter form invalid partition, this filter should be added on origin plan - PredicateAddContext predicateAddContext = new PredicateAddContext(partitionOnOriginPlan, partitionColumn); - Plan queryPlanWithUnionFilter = queryPlan.accept(new PredicateAdder(), - predicateAddContext); + PredicateAddContext predicateAddContext = new PredicateAddContext(null, partitionOnBaseTableMap); + Plan queryPlanWithUnionFilter = queryPlan.accept(new PredicateAdder(), predicateAddContext); if (!predicateAddContext.isHandleSuccess()) { return null; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index 8411bf14085e21..3ec8aa36f3b363 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -30,6 +30,7 @@ import org.apache.doris.common.UserException; import org.apache.doris.datasource.ExternalTable; import org.apache.doris.datasource.mvcc.MvccUtil; +import org.apache.doris.mtmv.BaseColInfo; import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVRelatedTableIf; import org.apache.doris.nereids.StatementContext; @@ -43,7 +44,6 @@ import org.apache.doris.nereids.trees.expressions.IsNull; import org.apache.doris.nereids.trees.expressions.LessThan; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; @@ -298,17 +298,18 @@ public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, catalogRelation); } } - if (predicates.getPartition() != null && predicates.getPartitionName() != null) { + if (predicates.getPartitions() != null) { if (!(table instanceof MTMVRelatedTableIf)) { return catalogRelation; } - for (Map.Entry> filterTableEntry : predicates.getPartition().entrySet()) { - if (!Objects.equals(new BaseTableInfo(table), filterTableEntry.getKey())) { + for (Map.Entry> filterTableEntry : predicates.getPartitions().entrySet()) { + BaseColInfo relatedTableColumnInfo = filterTableEntry.getKey(); + if (!Objects.equals(new BaseTableInfo(table), relatedTableColumnInfo.getTableInfo())) { continue; } Slot partitionSlot = null; for (Slot slot : catalogRelation.getOutput()) { - if (((SlotReference) slot).getName().equals(predicates.getPartitionName())) { + if (slot.getName().equals(relatedTableColumnInfo.getColName())) { partitionSlot = slot; break; } @@ -321,17 +322,26 @@ public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, Set partitionHasDataItems = new HashSet<>(); MTMVRelatedTableIf targetTable = (MTMVRelatedTableIf) table; for (String partitionName : filterTableEntry.getValue()) { - Partition partition = targetTable.getPartition(partitionName); - if (targetTable instanceof OlapTable && !((OlapTable) targetTable).selectNonEmptyPartitionIds( - Lists.newArrayList(partition.getId())).isEmpty()) { - // Add filter only when partition has data when olap table - partitionHasDataItems.add( - ((OlapTable) targetTable).getPartitionInfo().getItem(partition.getId())); + if (targetTable instanceof OlapTable) { + Partition partition = targetTable.getPartition(partitionName); + if (partition == null) { + // partition maybe deleted, skip it + continue; + } + if (!((OlapTable) targetTable).selectNonEmptyPartitionIds( + Lists.newArrayList(partition.getId())).isEmpty()) { + // Add filter only when partition has data when olap table + partitionHasDataItems.add( + ((OlapTable) targetTable).getPartitionInfo().getItem(partition.getId())); + } } if (targetTable instanceof ExternalTable) { + PartitionItem partitionItem = ((ExternalTable) targetTable).getNameToPartitionItems( + MvccUtil.getSnapshotFromContext(targetTable)).get(partitionName); // Add filter only when partition has data when external table - partitionHasDataItems.add(((ExternalTable) targetTable).getNameToPartitionItems( - MvccUtil.getSnapshotFromContext(targetTable)).get(partitionName)); + if (partitionItem != null) { + partitionHasDataItems.add(partitionItem); + } } } if (partitionHasDataItems.isEmpty()) { @@ -358,42 +368,27 @@ public Plan visitLogicalCatalogRelation(LogicalCatalogRelation catalogRelation, public static class PredicateAddContext { private final Map> predicates; - private final Map> partition; - private final String partitionName; + private final Map> partitions; private boolean handleSuccess = true; // when add filter by partition, if partition has no data, doesn't need to add filter. should be false private boolean needAddFilter = true; - public PredicateAddContext(Map> predicates) { - this(predicates, null, null); - } - - public PredicateAddContext(Map> partition, - String partitionName) { - this(null, partition, partitionName); - } - - public PredicateAddContext(Map> predicates, Map> partition, - String partitionName) { + public PredicateAddContext(Map> predicates, + Map> partitions) { this.predicates = predicates; - this.partition = partition; - this.partitionName = partitionName; + this.partitions = partitions; } public Map> getPredicates() { return predicates; } - public Map> getPartition() { - return partition; - } - - public String getPartitionName() { - return partitionName; + public Map> getPartitions() { + return partitions; } public boolean isEmpty() { - return predicates == null && partition == null; + return predicates == null && partitions == null; } public boolean isHandleSuccess() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java index 9ddced16b6ac20..32b8c24337d77e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/CreateMTMVInfo.java @@ -21,6 +21,7 @@ import org.apache.doris.analysis.ListPartitionDesc; import org.apache.doris.analysis.PartitionDesc; import org.apache.doris.analysis.RangePartitionDesc; +import org.apache.doris.catalog.Column; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.PartitionType; @@ -217,15 +218,27 @@ public void analyzeQuery(ConnectContext ctx) throws UserException { this.relation = mtmvAnalyzeQueryInfo.getRelation(); } + private List getPartitionColumn(String partitionColumnName) { + for (ColumnDefinition columnDefinition : columns) { + if (columnDefinition.getName().equalsIgnoreCase(partitionColumnName)) { + // current only support one partition col + return Lists.newArrayList(columnDefinition.translateToCatalogStyle()); + } + } + throw new AnalysisException("can not find partition column"); + } + private PartitionDesc generatePartitionDesc(ConnectContext ctx) { if (mvPartitionInfo.getPartitionType() == MTMVPartitionType.SELF_MANAGE) { return null; } - MTMVRelatedTableIf relatedTable = MTMVUtil.getRelatedTable(mvPartitionInfo.getRelatedTableInfo()); + // all pct table partition type is same + MTMVRelatedTableIf relatedTable = MTMVUtil.getRelatedTable(mvPartitionInfo.getPctInfos().get(0).getTableInfo()); List allPartitionDescs = null; try { allPartitionDescs = MTMVPartitionUtil - .getPartitionDescsByRelatedTable(properties, mvPartitionInfo, mvProperties); + .getPartitionDescsByRelatedTable(properties, mvPartitionInfo, mvProperties, + getPartitionColumn(mvPartitionInfo.getPartitionCol())); } catch (org.apache.doris.common.AnalysisException e) { throw new AnalysisException(e.getMessage(), e); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java index 8624bed9ceefae..5001e4cf4e67fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/MTMVPartitionDefinition.java @@ -26,7 +26,9 @@ import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.StringLiteral; import org.apache.doris.catalog.PartitionType; -import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.mvcc.MvccUtil; +import org.apache.doris.mtmv.BaseColInfo; +import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.mtmv.MTMVPartitionExprFactory; import org.apache.doris.mtmv.MTMVPartitionInfo; import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; @@ -38,18 +40,18 @@ import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; -import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.RelatedTableInfo; +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo; +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo.RelatedTableColumnInfo; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import com.google.common.collect.Sets; +import com.google.common.collect.Lists; import java.util.List; import java.util.Optional; -import java.util.Set; import java.util.stream.Collectors; /** @@ -90,21 +92,7 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl timeUnit = null; } mtmvPartitionInfo.setPartitionCol(partitionColName); - RelatedTableInfo relatedTableInfo = getRelatedTableInfo(planner, partitionColName, timeUnit); - mtmvPartitionInfo.setRelatedCol(relatedTableInfo.getColumn()); - mtmvPartitionInfo.setRelatedTable(relatedTableInfo.getTableInfo()); - if (relatedTableInfo.getPartitionExpression().isPresent()) { - // Set mv partition expr by relatedTableInfo, this is used for partition rollup and so on - if (relatedTableInfo.getPartitionExpression().get().getExpressionName() - .equalsIgnoreCase(PARTITION_BY_FUNCTION_NAME)) { - DateTrunc dateTrunc = (DateTrunc) relatedTableInfo.getPartitionExpression().get(); - // todo use new expression? - mtmvPartitionInfo.setExpr(new FunctionCallExpr(dateTrunc.getName(), - new FunctionParams(convertToLegacyArguments(dateTrunc.children())))); - mtmvPartitionInfo.setPartitionType(MTMVPartitionType.EXPR); - this.partitionType = MTMVPartitionType.EXPR; - } - } + fillPctInfos(planner, partitionColName, timeUnit, mtmvPartitionInfo); if (this.partitionType == MTMVPartitionType.EXPR) { try { MTMVPartitionExprFactory.getExprService(mtmvPartitionInfo.getExpr()).analyze(mtmvPartitionInfo); @@ -116,31 +104,55 @@ public MTMVPartitionInfo analyzeAndTransferToMTMVPartitionInfo(NereidsPlanner pl } // Should use rewritten plan without view and subQuery to get related partition table - private RelatedTableInfo getRelatedTableInfo(NereidsPlanner planner, String partitionColName, String timeUnit) { + private void fillPctInfos(NereidsPlanner planner, String partitionColName, + String timeUnit, MTMVPartitionInfo mtmvPartitionInfo) { CascadesContext cascadesContext = planner.getCascadesContext(); - RelatedTableInfo relatedTableInfo = MaterializedViewUtils - .getRelatedTableInfo(partitionColName, timeUnit, planner.getRewrittenPlan(), cascadesContext); + .getRelatedTableInfos(partitionColName, timeUnit, planner.getRewrittenPlan(), cascadesContext); if (!relatedTableInfo.isPctPossible()) { throw new AnalysisException(String.format("Unable to find a suitable base table for partitioning," + " the fail reason is %s", relatedTableInfo.getFailReason())); } - MTMVRelatedTableIf mtmvBaseRelatedTable = MTMVUtil.getRelatedTable(relatedTableInfo.getTableInfo()); - Set partitionColumnNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER); - try { - partitionColumnNames.addAll(mtmvBaseRelatedTable.getPartitionColumnNames(Optional.empty())); - } catch (DdlException e) { - throw new AnalysisException(e.getMessage(), e); - } + List tableColumnInfos = relatedTableInfo.getTableColumnInfos(); + List pctInfos = Lists.newArrayList(); + for (RelatedTableColumnInfo tableColumnInfo : tableColumnInfos) { + String columnStr = tableColumnInfo.getColumnStr(); + BaseTableInfo tableInfo = tableColumnInfo.getTableInfo(); + BaseColInfo baseColInfo = new BaseColInfo(columnStr, tableInfo); + pctInfos.add(baseColInfo); + Optional partitionExpression = tableColumnInfo.getPartitionExpression(); + if (partitionExpression.isPresent() && partitionExpression.get().getExpressionName() + .equalsIgnoreCase(PARTITION_BY_FUNCTION_NAME)) { + DateTrunc dateTrunc = (DateTrunc) partitionExpression.get(); + mtmvPartitionInfo.setExpr(new FunctionCallExpr(dateTrunc.getName(), + new FunctionParams(convertToLegacyArguments(dateTrunc.children())))); + mtmvPartitionInfo.setPartitionType(MTMVPartitionType.EXPR); + this.partitionType = MTMVPartitionType.EXPR; + } - if (!partitionColumnNames.contains(relatedTableInfo.getColumn())) { - throw new AnalysisException("error related column: " + relatedTableInfo.getColumn()); } - if (!(mtmvBaseRelatedTable.getPartitionType(Optional.empty()).equals(PartitionType.LIST)) - && partitionColumnNames.size() != 1) { - throw new AnalysisException("only List PartitionType support multi column partition."); + if (pctInfos.isEmpty()) { + throw new AnalysisException( + "Unable to find a suitable base table for partitioning,the fail reason is pctInfosSet.size() is 0"); + } + MTMVRelatedTableIf relatedTable = MTMVUtil.getRelatedTable(pctInfos.get(0).getTableInfo()); + PartitionType relatedTablePartitionType = relatedTable.getPartitionType( + MvccUtil.getSnapshotFromContext(relatedTable)); + if (pctInfos.size() > 1) { + // check all partition type of pct table is same + for (BaseColInfo baseColInfo : pctInfos) { + MTMVRelatedTableIf pctTable = MTMVUtil.getRelatedTable(baseColInfo.getTableInfo()); + PartitionType partitionType = pctTable.getPartitionType( + MvccUtil.getSnapshotFromContext(pctTable)); + if (!partitionType.equals(relatedTablePartitionType)) { + throw new AnalysisException("partition type of multi pctTables must be same, pctInfos:" + pctInfos); + } + } } - return relatedTableInfo; + // for compatible + mtmvPartitionInfo.setRelatedCol(pctInfos.get(0).getColName()); + mtmvPartitionInfo.setRelatedTable(pctInfos.get(0).getTableInfo()); + mtmvPartitionInfo.setPctInfos(pctInfos); } private static List convertToLegacyArguments(List children) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java index 096bccac6822f9..a4f37655eee225 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/info/RefreshMTMVInfo.java @@ -86,8 +86,9 @@ public void analyze(ConnectContext ctx) { } private void checkPartitionExist(MTMV mtmv) throws org.apache.doris.common.AnalysisException { - MTMVRelatedTableIf relatedTable = mtmv.getMvPartitionInfo().getRelatedTable(); - List tables = Lists.newArrayList(mtmv, relatedTable); + Set pctTables = mtmv.getMvPartitionInfo().getPctTables(); + List tables = Lists.newArrayList(pctTables); + tables.add(mtmv); tables.sort(Comparator.comparing(TableIf::getId)); MetaLockUtils.readLockTables(tables); try { @@ -97,7 +98,8 @@ private void checkPartitionExist(MTMV mtmv) throws org.apache.doris.common.Analy + "does not support refreshing by partition"); } List partitionDescs = MTMVPartitionUtil.getPartitionDescsByRelatedTable( - mtmv.getTableProperty().getProperties(), mtmv.getMvPartitionInfo(), mtmv.getMvProperties()); + mtmv.getTableProperty().getProperties(), mtmv.getMvPartitionInfo(), mtmv.getMvProperties(), + mtmv.getPartitionColumns()); Set shouldExistPartitionNames = Sets.newHashSetWithExpectedSize(partitionDescs.size()); partitionDescs.stream().forEach(desc -> { shouldExistPartitionNames.add(((SinglePartitionDesc) desc).getPartitionName()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionCheckUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionCheckUtilTest.java index 224816de4ae6f4..b50a3fcb18b708 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionCheckUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionCheckUtilTest.java @@ -234,9 +234,13 @@ public void testCompareDynamicPartitionNotEqual() throws AnalysisException { } @Test - public void testCompareAutpPartition() throws AnalysisException { + public void testCompareAutoPartition() throws AnalysisException { new Expectations() { { + relatedPartitionInfo.enableAutomaticPartition(); + minTimes = 0; + result = true; + partitionExprUtil.getFunctionIntervalInfo(originalExprs, (PartitionType) any); minTimes = 0; result = partitionExprUtil.new FunctionIntervalInfo("datetrunc", "week", 1); @@ -251,7 +255,7 @@ public void testCompareAutpPartition() throws AnalysisException { } @Test - public void testCompareAutpPartitionNotEqual() throws AnalysisException { + public void testCompareAutoPartitionNotEqual() throws AnalysisException { new Expectations() { { partitionExprUtil.getFunctionIntervalInfo(originalExprs, (PartitionType) any); diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java index e0d5e1b3b9269b..5fd7f15bf728a6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVPartitionUtilTest.java @@ -149,11 +149,12 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc minTimes = 0; result = baseSnapshotIf; - refreshSnapshot.equalsWithRelatedPartition(anyString, anyString, (MTMVSnapshotIf) any); + refreshSnapshot.equalsWithPct(anyString, anyString, (MTMVSnapshotIf) any, + (BaseTableInfo) any); minTimes = 0; result = true; - refreshSnapshot.getSnapshotPartitions(anyString); + refreshSnapshot.getPctSnapshots(anyString, (BaseTableInfo) any); minTimes = 0; result = Sets.newHashSet("name2"); @@ -206,7 +207,7 @@ public void testIsMTMVSyncNotSync() { @Test public void testIsSyncWithPartition() throws AnalysisException { boolean isSyncWithPartition = MTMVPartitionUtil - .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2")); + .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2"), baseOlapTable); Assert.assertTrue(isSyncWithPartition); } @@ -214,13 +215,13 @@ public void testIsSyncWithPartition() throws AnalysisException { public void testIsSyncWithPartitionNotEqual() throws AnalysisException { new Expectations() { { - refreshSnapshot.getSnapshotPartitions(anyString); + refreshSnapshot.getPctSnapshots(anyString, (BaseTableInfo) any); minTimes = 0; result = Sets.newHashSet("name2", "name3"); } }; boolean isSyncWithPartition = MTMVPartitionUtil - .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2")); + .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2"), baseOlapTable); Assert.assertFalse(isSyncWithPartition); } @@ -228,13 +229,14 @@ public void testIsSyncWithPartitionNotEqual() throws AnalysisException { public void testIsSyncWithPartitionNotSync() throws AnalysisException { new Expectations() { { - refreshSnapshot.equalsWithRelatedPartition(anyString, anyString, (MTMVSnapshotIf) any); + refreshSnapshot.equalsWithPct(anyString, anyString, (MTMVSnapshotIf) any, + (BaseTableInfo) any); minTimes = 0; result = false; } }; boolean isSyncWithPartition = MTMVPartitionUtil - .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2")); + .isSyncWithPartitions(context, "name1", Sets.newHashSet("name2"), baseOlapTable); Assert.assertFalse(isSyncWithPartition); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java index 3e61a7ca54aa1f..f81c77ee75f3c4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRefreshSnapshotTest.java @@ -83,7 +83,7 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc Map partitionSnapshots = Maps.newHashMap(); MTMVRefreshPartitionSnapshot mvp1PartitionSnapshot = new MTMVRefreshPartitionSnapshot(); partitionSnapshots.put(mvExistPartitionName, mvp1PartitionSnapshot); - mvp1PartitionSnapshot.getPartitions().put(relatedExistPartitionName, p1Snapshot); + mvp1PartitionSnapshot.getPctSnapshot(existTable).put(relatedExistPartitionName, p1Snapshot); mvp1PartitionSnapshot.addTableSnapshot(existTable, t1Snapshot); refreshSnapshot.updateSnapshots(partitionSnapshots, Sets.newHashSet(mvExistPartitionName)); } @@ -91,24 +91,25 @@ public void setUp() throws NoSuchMethodException, SecurityException, AnalysisExc @Test public void testPartitionSync() { // normal - boolean sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, - new MTMVVersionSnapshot(correctVersion, 0)); + boolean sync = refreshSnapshot.equalsWithPct(mvExistPartitionName, relatedExistPartitionName, + new MTMVVersionSnapshot(correctVersion, 0), existTable); Assert.assertTrue(sync); // non exist mv partition - sync = refreshSnapshot.equalsWithRelatedPartition("mvp2", relatedExistPartitionName, - new MTMVVersionSnapshot(correctVersion, 0)); + sync = refreshSnapshot.equalsWithPct("mvp2", relatedExistPartitionName, + new MTMVVersionSnapshot(correctVersion, 0), existTable); Assert.assertFalse(sync); // non exist related partition sync = refreshSnapshot - .equalsWithRelatedPartition(mvExistPartitionName, "p2", new MTMVVersionSnapshot(correctVersion, 0)); + .equalsWithPct(mvExistPartitionName, "p2", new MTMVVersionSnapshot(correctVersion, 0), + existTable); Assert.assertFalse(sync); // snapshot value not equal - sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, - new MTMVVersionSnapshot(2L, 0)); + sync = refreshSnapshot.equalsWithPct(mvExistPartitionName, relatedExistPartitionName, + new MTMVVersionSnapshot(2L, 0), existTable); Assert.assertFalse(sync); // snapshot type not equal - sync = refreshSnapshot.equalsWithRelatedPartition(mvExistPartitionName, relatedExistPartitionName, - new MTMVTimestampSnapshot(correctVersion)); + sync = refreshSnapshot.equalsWithPct(mvExistPartitionName, relatedExistPartitionName, + new MTMVTimestampSnapshot(correctVersion), existTable); Assert.assertFalse(sync); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorTest.java new file mode 100644 index 00000000000000..d14c0f08f1a981 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescGeneratorTest.java @@ -0,0 +1,295 @@ +// 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.mtmv; + +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.FunctionCallExpr; +import org.apache.doris.analysis.PartitionKeyDesc; +import org.apache.doris.analysis.StringLiteral; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.util.PropertyAnalyzer; +import org.apache.doris.mtmv.MTMVPartitionInfo.MTMVPartitionType; +import org.apache.doris.utframe.TestWithFeService; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class MTMVRelatedPartitionDescGeneratorTest extends TestWithFeService { + + @Override + protected void runBeforeAll() throws Exception { + createDatabaseAndUse("test"); + + createTable("CREATE TABLE `t1` (`c1` date, `c2` int)\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY RANGE(c1) (PARTITION p20210201 VALUES [('2021-02-01'), ('2021-02-02'))," + + "PARTITION p20210202 VALUES [('2021-02-02'), ('2021-02-03'))," + + "PARTITION p20500203 VALUES [('2050-02-03'), ('2050-02-04'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t2` (`c1` int, `c2` VARCHAR(100))\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY List(c1,c2) (PARTITION p1_bj VALUES IN (('1','bj'))," + + "PARTITION p2_bj VALUES IN (('2','bj'))," + + "PARTITION p1_sh VALUES IN (('1','sh'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t3` (`c1` date, `c2` int)\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY RANGE(c1) (PARTITION p20210201 VALUES [('2021-02-01'), ('2021-02-03'))," + + "PARTITION p20500203 VALUES [('2050-02-03'), ('2050-02-04'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t4` (`c1` date, `c2` int)\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY RANGE(c1) (PARTITION p20500204 VALUES [('2050-02-04'), ('2050-02-05'))," + + "PARTITION p20500203 VALUES [('2050-02-03'), ('2050-02-04'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t5` (`c1` int, `c2` VARCHAR(100))\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY List(c1,c2) (PARTITION p1_bj VALUES IN (('1','bj'))," + + "PARTITION p2_bj VALUES IN (('2','bj'))," + + "PARTITION p1_sh VALUES IN (('1','sh'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t6` (`c1` int, `c2` VARCHAR(100))\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY List(c1) (PARTITION p1 VALUES IN (('1'))," + + "PARTITION p2 VALUES IN (('2'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + + createTable("CREATE TABLE `t7` (`c1` int, `c2` VARCHAR(100))\n" + + "ENGINE=OLAP\n" + + "DUPLICATE KEY(`c1`)\n" + + "PARTITION BY List(c1) (PARTITION p1_3 VALUES IN (('1'),('3'))," + + "PARTITION p2 VALUES IN (('2'))) distributed by hash(c1) " + + "buckets 1 properties('replication_num' = '1');"); + } + + @Test + public void testSimple() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 3 partition + Assertions.assertEquals(3, partitionKeyDescMap.size()); + OlapTable t1 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t1"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + // key is t1 + // value like p20210201 + Assertions.assertEquals(1, onePartitionMap.size()); + Assertions.assertEquals(1, onePartitionMap.get(t1).size()); + } + } + + @Test + public void testLimit() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + HashMap mvProperty = Maps.newHashMap(); + mvProperty.put(PropertyAnalyzer.PROPERTIES_PARTITION_SYNC_LIMIT, "1"); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, mvProperty, + Lists.newArrayList(c1Column)); + // 3 partition + Assertions.assertEquals(1, partitionKeyDescMap.size()); + OlapTable t1 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t1"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + // key is t1 + // value is p20210201 + Assertions.assertEquals(1, onePartitionMap.size()); + Assertions.assertEquals(1, onePartitionMap.get(t1).size()); + } + } + + @Test + public void testOneCol() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t2")); + Column c1Column = new Column("c1", PrimitiveType.INT); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 3 partition + Assertions.assertEquals(2, partitionKeyDescMap.size()); + OlapTable t2 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t2"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + // key is t1 + // value like p20210201 + Assertions.assertEquals(1, onePartitionMap.size()); + int partitionNum = onePartitionMap.get(t2).size(); + Assertions.assertTrue(partitionNum == 1 || partitionNum == 2); + } + } + + @Test + public void testDateTrunc() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1")); + mtmvPartitionInfo.setPartitionType(MTMVPartitionType.EXPR); + List params = new ArrayList<>(); + params.add(new StringLiteral("c1")); + params.add(new StringLiteral("month")); + FunctionCallExpr functionCallExpr = new FunctionCallExpr("date_trunc", params); + mtmvPartitionInfo.setExpr(functionCallExpr); + Column c1Column = new Column("c1", PrimitiveType.INT); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 2 partition + Assertions.assertEquals(2, partitionKeyDescMap.size()); + OlapTable t1 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t1"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + Assertions.assertEquals(1, onePartitionMap.size()); + int partitionNum = onePartitionMap.get(t1).size(); + Assertions.assertTrue(partitionNum == 1 || partitionNum == 2); + } + } + + @Test + public void testIntersect() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1", "t3")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Assertions.assertThrows(AnalysisException.class, + () -> MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column))); + } + + @Test + public void testIntersectList() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t6", "t7")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Assertions.assertThrows(AnalysisException.class, + () -> MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column))); + } + + @Test + public void testMultiPctTables() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1", "t4")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 4 partition + Assertions.assertEquals(4, partitionKeyDescMap.size()); + boolean hasOne = false; + boolean hasTwo = false; + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + if (onePartitionMap.size() == 1) { + hasOne = true; + } else if (onePartitionMap.size() == 2) { + hasTwo = true; + } else { + throw new RuntimeException("failed"); + } + } + Assertions.assertTrue(hasOne); + Assertions.assertTrue(hasTwo); + } + + @Test + public void testMultiList() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t2", "t5")); + Column c1Column = new Column("c1", PrimitiveType.INT); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 2 partition + Assertions.assertEquals(2, partitionKeyDescMap.size()); + OlapTable t2 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t2"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + // key is t1 + // value like p20210201 + Assertions.assertEquals(2, onePartitionMap.size()); + int partitionNum = onePartitionMap.get(t2).size(); + Assertions.assertTrue(partitionNum == 1 || partitionNum == 2); + } + } + + @Test + public void testMultiListComplex() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t2", "t6")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Map>> partitionKeyDescMap + = MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column)); + // 2 partition + Assertions.assertEquals(2, partitionKeyDescMap.size()); + OlapTable t6 = (OlapTable) Env.getCurrentEnv().getInternalCatalog().getDbOrAnalysisException("test") + .getTableOrAnalysisException("t6"); + for (Map> onePartitionMap : partitionKeyDescMap.values()) { + // key is t1 + // value like p20210201 + Assertions.assertEquals(2, onePartitionMap.size()); + int partitionNum = onePartitionMap.get(t6).size(); + Assertions.assertTrue(partitionNum == 1); + } + } + + @Test + public void testBothListAndRange() throws Exception { + MTMVPartitionInfo mtmvPartitionInfo = getMTMVPartitionInfo(Lists.newArrayList("t1", "t2")); + Column c1Column = new Column("c1", PrimitiveType.DATE); + Assertions.assertThrows(AnalysisException.class, + () -> MTMVPartitionUtil.generateRelatedPartitionDescs(mtmvPartitionInfo, Maps.newHashMap(), + Lists.newArrayList(c1Column))); + } + + private MTMVPartitionInfo getMTMVPartitionInfo(List pctTableNames) throws AnalysisException { + MTMVPartitionInfo mtmvPartitionInfo = new MTMVPartitionInfo(); + mtmvPartitionInfo.setPartitionType(MTMVPartitionType.FOLLOW_BASE_TABLE); + mtmvPartitionInfo.setPartitionCol("c1"); + List pctInfos = new ArrayList<>(); + for (String pctTableName : pctTableNames) { + OlapTable pctOlapTable = (OlapTable) Env.getCurrentEnv().getInternalCatalog() + .getDbOrAnalysisException("test") + .getTableOrAnalysisException(pctTableName); + BaseTableInfo pctTableInfo = new BaseTableInfo(pctOlapTable); + BaseColInfo pctColInfo = new BaseColInfo("c1", pctTableInfo); + pctInfos.add(pctColInfo); + } + mtmvPartitionInfo.setPctInfos(pctInfos); + return mtmvPartitionInfo; + } + +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGeneratorTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGeneratorTest.java index 15036c5e497516..1fcc3029e79f1f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGeneratorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRelatedPartitionDescRollUpGeneratorTest.java @@ -86,7 +86,7 @@ public void testRollUpRange() throws AnalysisException { relatedPartitionDescs.put(desc20200102, Sets.newHashSet("name2")); relatedPartitionDescs.put(desc20200201, Sets.newHashSet("name3")); Map> res = generator.rollUpRange(relatedPartitionDescs, - mtmvPartitionInfo); + mtmvPartitionInfo, null); PartitionKeyDesc expectDesc202001 = PartitionKeyDesc.createFixed( Lists.newArrayList(new PartitionValue("2020-01-01")), diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java index 23b960c7dcbad8..687f171be408ef 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVRewriteUtilTest.java @@ -17,16 +17,30 @@ package org.apache.doris.mtmv; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.MTMV; import org.apache.doris.catalog.Partition; +import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.PartitionType; +import org.apache.doris.catalog.TableIndexes; import org.apache.doris.common.AnalysisException; +import org.apache.doris.common.DdlException; +import org.apache.doris.common.Pair; +import org.apache.doris.datasource.mvcc.MvccSnapshot; import org.apache.doris.info.TableNameInfo; import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVRefreshState; import org.apache.doris.mtmv.MTMVRefreshEnum.MTMVState; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.statistics.AnalysisInfo; +import org.apache.doris.statistics.BaseAnalysisTask; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.thrift.TTableDescriptor; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import mockit.Expectations; import mockit.Mocked; @@ -34,7 +48,13 @@ import org.junit.Before; import org.junit.Test; +import java.io.DataOutput; +import java.io.IOException; import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; import java.util.Set; public class MTMVRewriteUtilTest { @@ -323,4 +343,245 @@ public void testGetMTMVCanRewritePartitionsRefreshStateInit() { null); Assert.assertEquals(0, mtmvCanRewritePartitions.size()); } + + @Test + public void testPctToMv() { + TestMTMVRelatedTable t1 = new TestMTMVRelatedTable("t1"); + TestMTMVRelatedTable t2 = new TestMTMVRelatedTable("t2"); + Map>> partitionMappings = Maps.newHashMap(); + partitionMappings.put("mv_p1", + ImmutableMap.of(t1, Sets.newHashSet("t1_p1", "t1_p2"), t2, Sets.newHashSet("t2_p1"))); + partitionMappings.put("mv_p2", ImmutableMap.of(t2, Sets.newHashSet("t2_p2"))); + Map, String> pctToMv = MTMVRewriteUtil.getPctToMv(partitionMappings); + Assert.assertEquals("mv_p1", pctToMv.get(Pair.of(t1, "t1_p1"))); + Assert.assertEquals("mv_p1", pctToMv.get(Pair.of(t1, "t1_p2"))); + Assert.assertEquals("mv_p1", pctToMv.get(Pair.of(t2, "t2_p1"))); + Assert.assertEquals("mv_p2", pctToMv.get(Pair.of(t2, "t2_p2"))); + } + + private static class TestMTMVRelatedTable implements MTMVRelatedTableIf { + + private String name; + + public TestMTMVRelatedTable(String name) { + this.name = name; + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + TestMTMVRelatedTable that = (TestMTMVRelatedTable) o; + return Objects.equals(name, that.name); + } + + @Override + public int hashCode() { + return Objects.hashCode(name); + } + + @Override + public String toString() { + return name; + } + + @Override + public Map getAndCopyPartitionItems(Optional snapshot) + throws AnalysisException { + return null; + } + + @Override + public PartitionType getPartitionType(Optional snapshot) { + return null; + } + + @Override + public Set getPartitionColumnNames(Optional snapshot) throws DdlException { + return null; + } + + @Override + public List getPartitionColumns(Optional snapshot) { + return null; + } + + @Override + public MTMVSnapshotIf getPartitionSnapshot(String partitionName, MTMVRefreshContext context, + Optional snapshot) throws AnalysisException { + return null; + } + + @Override + public MTMVSnapshotIf getTableSnapshot(MTMVRefreshContext context, Optional snapshot) + throws AnalysisException { + return null; + } + + @Override + public MTMVSnapshotIf getTableSnapshot(Optional snapshot) throws AnalysisException { + return null; + } + + @Override + public long getNewestUpdateVersionOrTime() { + return 0; + } + + @Override + public boolean isPartitionColumnAllowNull() { + return false; + } + + @Override + public long getId() { + return 0; + } + + @Override + public String getName() { + return ""; + } + + @Override + public TableType getType() { + return null; + } + + @Override + public List getFullSchema() { + return null; + } + + @Override + public List getBaseSchema() { + return null; + } + + @Override + public List getBaseSchema(boolean full) { + return null; + } + + @Override + public void setNewFullSchema(List newSchema) { + + } + + @Override + public Column getColumn(String name) { + return null; + } + + @Override + public String getMysqlType() { + return ""; + } + + @Override + public String getEngine() { + return ""; + } + + @Override + public String getComment() { + return ""; + } + + @Override + public long getCreateTime() { + return 0; + } + + @Override + public long getUpdateTime() { + return 0; + } + + @Override + public long getRowCount() { + return 0; + } + + @Override + public long getCachedRowCount() { + return 0; + } + + @Override + public long fetchRowCount() { + return 0; + } + + @Override + public long getDataLength() { + return 0; + } + + @Override + public long getAvgRowLength() { + return 0; + } + + @Override + public long getIndexLength() { + return 0; + } + + @Override + public long getLastCheckTime() { + return 0; + } + + @Override + public String getComment(boolean escapeQuota) { + return ""; + } + + @Override + public TTableDescriptor toThrift() { + return null; + } + + @Override + public BaseAnalysisTask createAnalysisTask(AnalysisInfo info) { + return null; + } + + @Override + public DatabaseIf getDatabase() { + return null; + } + + @Override + public Optional getColumnStatistic(String colName) { + return Optional.empty(); + } + + @Override + public Set> getColumnIndexPairs(Set columns) { + return null; + } + + @Override + public List getChunkSizes() { + return null; + } + + @Override + public void write(DataOutput out) throws IOException { + + } + + @Override + public boolean autoAnalyzeEnabled() { + return false; + } + + @Override + public TableIndexes getTableIndexes() { + return null; + } + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java index b0187dba9ccc1a..8074ce99f81972 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/mtmv/MTMVTest.java @@ -62,8 +62,8 @@ public void testToInfoString() { + "AbstractTask{jobId=null, taskId=1, status=null, createTimeMs=null, startTimeMs=null, " + "finishTimeMs=null, taskType=null, errMsg='null'}]}, mvProperties={}, " + "relation=MTMVRelation{baseTables=[], baseTablesOneLevel=[], baseViews=[]}, " - + "mvPartitionInfo=MTMVPartitionInfo{partitionType=null, relatedTable=null, " - + "relatedCol='null', partitionCol='null', expr='null'}, " + + "mvPartitionInfo=MTMVPartitionInfo{partitionType=null, pctInfos=[], " + + "partitionCol='null', expr='null'}, " + "refreshSnapshot=MTMVRefreshSnapshot{partitionSnapshots={}}, id=1, name='null', " + "qualifiedDbName='db1', comment='comment1'}"; MTMV mtmv = new MTMV(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java index c3f38697d4f964..d1ba92c7d1cdcf 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/StructInfoMapTest.java @@ -34,6 +34,8 @@ import java.util.BitSet; import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -67,7 +69,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) { return true; } }; @@ -138,7 +140,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) { return true; } }; @@ -196,7 +198,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean forceConsistent, - Set relatedPartitions) { + Map, Set> queryUsedPartitions) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java index 20a02cfd84cdeb..e60ab621ffe3d4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/IdStatisticsMapTest.java @@ -35,6 +35,7 @@ import org.junit.jupiter.api.Test; import java.util.BitSet; +import java.util.List; import java.util.Map; import java.util.Set; @@ -56,7 +57,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, - Set queryUsedRelatedTablePartitionsMap) { + Map, Set> queryUsedPartitions) { return true; } }; @@ -113,7 +114,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, - Set queryUsedRelatedTablePartitionsMap) { + Map, Set> queryUsedPartitions) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java index c82e1310cbeb7a..615107bf99250f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/MvTableIdIsLongTest.java @@ -31,6 +31,8 @@ import org.junit.jupiter.api.Test; import java.util.BitSet; +import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -51,7 +53,7 @@ public BitSet getDisableNereidsRules() { new MockUp() { @Mock public boolean isMVPartitionValid(MTMV mtmv, ConnectContext ctx, boolean isMVPartitionValid, - Set queryUsedRelatedTablePartitionsMap) { + Map, Set> queryUsedPartitions) { return true; } }; diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java index 685cfcb6177fd2..ae225aec5e9a3c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewUtilsTest.java @@ -20,7 +20,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.mtmv.BaseTableInfo; -import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.RelatedTableInfo; +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo.RelatedTableColumnInfo; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.utframe.TestWithFeService; @@ -263,7 +263,7 @@ public void joinPartitionNameSameTest() { PlanChecker.from(connectContext) .checkExplain("select t1.upgrade_day, t2.batch_no, count(*) " + "from test2 t2 join test1 t1 on " - + "t1.upgrade_day = t2.upgrade_day " + + "t1.vin_type1 = t2.vin_type2 " + "group by t1.upgrade_day, t2.batch_no;", nereidsPlanner -> { Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); @@ -461,7 +461,7 @@ public void getRelatedTableInfoSelfJoinTest() { MaterializedViewUtils.getRelatedTableInfo("l_orderkey", null, rewrittenPlan, nereidsPlanner.getCascadesContext()); Assertions.assertTrue(relatedTableInfo.getFailReason().contains( - "self join doesn't support partition update")); + "partition column is in join invalid side, but is not in join condition")); Assertions.assertFalse(relatedTableInfo.isPctPossible()); }); @@ -477,7 +477,7 @@ public void getRelatedTableInfoSelfJoinTest() { MaterializedViewUtils.getRelatedTableInfo("l_orderkey", null, rewrittenPlan, nereidsPlanner.getCascadesContext()); Assertions.assertTrue(relatedTableInfo.getFailReason().contains( - "self join doesn't support partition update")); + "partition column is in invalid catalog relation to check")); Assertions.assertFalse(relatedTableInfo.isPctPossible()); }); @@ -574,7 +574,7 @@ public void getRelatedTableInfoTestWithoutPartitionTest() { MaterializedViewUtils.getRelatedTableInfo("PS_SUPPLYCOST", null, rewrittenPlan, nereidsPlanner.getCascadesContext()); Assertions.assertTrue(relatedTableInfo.getFailReason().contains( - "self join doesn't support partition update")); + "related base table is not partition table")); Assertions.assertFalse(relatedTableInfo.isPctPossible()); }); } @@ -711,6 +711,37 @@ public void testPartitionDateTruncShouldNotTrack() { }); } + @Test + public void testPartitionDateTruncShouldTrack1() { + PlanChecker.from(connectContext) + .checkExplain("SELECT date_trunc(t1.L_SHIPDATE, 'day') as date_alias, t2.O_ORDERDATE, t1.L_QUANTITY, t2.O_ORDERSTATUS, " + + "count(distinct case when t1.L_SUPPKEY > 0 then t2.O_ORDERSTATUS else null end) as cnt_1 " + + "from " + + " (select * from " + + " lineitem " + + " where L_SHIPDATE in ('2017-01-30')) t1 " + + "left join " + + " (select * from " + + " orders " + + " where O_ORDERDATE in ('2017-01-30')) t2 " + + "on t1.L_ORDERKEY = t2.O_ORDERKEY " + + "group by " + + "t1.L_SHIPDATE, " + + "t2.O_ORDERDATE, " + + "t1.L_QUANTITY, " + + "t2.O_ORDERSTATUS;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfo("date_alias", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + checkRelatedTableInfo(relatedTableInfo, + "lineitem", + "L_SHIPDATE", + true); + }); + } + @Test public void testPartitionDateTruncShouldTrack() { PlanChecker.from(connectContext) @@ -839,10 +870,9 @@ public void getRelatedTableInfoWhenMultiBaseTablePartition() { RelatedTableInfo relatedTableInfo = MaterializedViewUtils.getRelatedTableInfo("upgrade_day", null, rewrittenPlan, nereidsPlanner.getCascadesContext()); - checkRelatedTableInfo(relatedTableInfo, - "test1", - "upgrade_day", - true); + Assertions.assertTrue(relatedTableInfo.getFailReason().contains( + "partition column is not in group by or window partition by")); + Assertions.assertFalse(relatedTableInfo.isPctPossible()); }); } @@ -927,7 +957,10 @@ private void checkRelatedTableInfo(RelatedTableInfo relatedTableInfo, String expectColumnName, boolean pctPossible) { Assertions.assertNotNull(relatedTableInfo); - BaseTableInfo relatedBaseTableInfo = relatedTableInfo.getTableInfo(); + Assertions.assertTrue(pctPossible); + + RelatedTableColumnInfo columnInfo = relatedTableInfo.getTableColumnInfos().get(0); + BaseTableInfo relatedBaseTableInfo = columnInfo.getTableInfo(); try { TableIf tableIf = Env.getCurrentEnv().getCatalogMgr() .getCatalogOrAnalysisException(relatedBaseTableInfo.getCtlId()) @@ -937,7 +970,6 @@ private void checkRelatedTableInfo(RelatedTableInfo relatedTableInfo, } catch (Exception exception) { Assertions.fail(); } - Assertions.assertEquals(relatedTableInfo.getColumn().toLowerCase(), expectColumnName.toLowerCase()); - Assertions.assertTrue(pctPossible); + Assertions.assertEquals(columnInfo.getColumnStr().toLowerCase(), expectColumnName.toLowerCase()); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionColumnTraceTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionColumnTraceTest.java new file mode 100644 index 00000000000000..caac4e63d4058d --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/exploration/mv/PartitionColumnTraceTest.java @@ -0,0 +1,1165 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.rules.exploration.mv.RelatedTableInfo.RelatedTableColumnInfo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DateTrunc; +import org.apache.doris.nereids.trees.plans.Plan; +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 org.apache.commons.lang3.StringUtils; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +public class PartitionColumnTraceTest extends TestWithFeService { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("partition_column_trace_test"); + useDatabase("partition_column_trace_test"); + + createTable("CREATE TABLE IF NOT EXISTS lineitem (\n" + + " L_ORDERKEY INTEGER NOT NULL,\n" + + " L_PARTKEY INTEGER NOT NULL,\n" + + " L_SUPPKEY INTEGER NOT NULL,\n" + + " L_LINENUMBER INTEGER NOT NULL,\n" + + " L_QUANTITY DECIMALV3(15,2) NOT NULL,\n" + + " L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL,\n" + + " L_DISCOUNT DECIMALV3(15,2) NOT NULL,\n" + + " L_TAX DECIMALV3(15,2) NOT NULL,\n" + + " L_RETURNFLAG CHAR(1) NOT NULL,\n" + + " L_LINESTATUS CHAR(1) NOT NULL,\n" + + " L_SHIPDATE DATE NOT NULL,\n" + + " L_COMMITDATE DATE NOT NULL,\n" + + " L_RECEIPTDATE DATE NOT NULL,\n" + + " L_SHIPINSTRUCT CHAR(25) NOT NULL,\n" + + " L_SHIPMODE CHAR(10) NOT NULL,\n" + + " L_COMMENT VARCHAR(44) NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)\n" + + "PARTITION BY RANGE(L_SHIPDATE) (PARTITION `day_1` VALUES LESS THAN ('2017-02-01'))\n" + + "DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"); + + createTable("CREATE TABLE IF NOT EXISTS orders (\n" + + " O_ORDERKEY INTEGER NOT NULL,\n" + + " O_CUSTKEY INTEGER NOT NULL,\n" + + " O_ORDERSTATUS CHAR(1) NOT NULL,\n" + + " O_TOTALPRICE DECIMALV3(15,2) NOT NULL,\n" + + " O_ORDERDATE DATE NOT NULL,\n" + + " O_ORDERDATE_NOT DATE NOT NULL,\n" + + " O_ORDERPRIORITY CHAR(15) NOT NULL, \n" + + " O_CLERK CHAR(15) NOT NULL, \n" + + " O_SHIPPRIORITY INTEGER NOT NULL,\n" + + " O_COMMENT VARCHAR(79) NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)\n" + + "PARTITION BY RANGE(O_ORDERDATE) (PARTITION `day_2` VALUES LESS THAN ('2017-03-01'))\n" + + "DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"); + + createTable("CREATE TABLE IF NOT EXISTS orders_no_part (\n" + + " O_ORDERKEY INTEGER NOT NULL,\n" + + " O_CUSTKEY INTEGER NOT NULL,\n" + + " O_ORDERSTATUS CHAR(1) NOT NULL,\n" + + " O_TOTALPRICE DECIMALV3(15,2) NOT NULL,\n" + + " O_ORDERDATE DATE NOT NULL,\n" + + " O_ORDERDATE_NOT DATE NOT NULL,\n" + + " O_ORDERPRIORITY CHAR(15) NOT NULL, \n" + + " O_CLERK CHAR(15) NOT NULL, \n" + + " O_SHIPPRIORITY INTEGER NOT NULL,\n" + + " O_COMMENT VARCHAR(79) NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)\n" + + "DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"); + + createTable("CREATE TABLE IF NOT EXISTS lineitem_auto (\n" + + " L_ORDERKEY INTEGER NOT NULL,\n" + + " L_PARTKEY INTEGER NOT NULL,\n" + + " L_SUPPKEY INTEGER NOT NULL,\n" + + " L_LINENUMBER INTEGER NOT NULL,\n" + + " L_QUANTITY DECIMALV3(15,2) NOT NULL,\n" + + " L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL,\n" + + " L_DISCOUNT DECIMALV3(15,2) NOT NULL,\n" + + " L_TAX DECIMALV3(15,2) NOT NULL,\n" + + " L_RETURNFLAG CHAR(1) NOT NULL,\n" + + " L_LINESTATUS CHAR(1) NOT NULL,\n" + + " L_SHIPDATE DATE NOT NULL,\n" + + " L_COMMITDATE DATE NOT NULL,\n" + + " L_RECEIPTDATE DATE NOT NULL,\n" + + " L_SHIPINSTRUCT CHAR(25) NOT NULL,\n" + + " L_SHIPMODE CHAR(10) NOT NULL,\n" + + " L_COMMENT VARCHAR(44) NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(L_ORDERKEY, L_PARTKEY, L_SUPPKEY, L_LINENUMBER)\n" + + "auto partition by range (date_trunc(`L_SHIPDATE`, 'day')) ()\n" + + "DISTRIBUTED BY HASH(L_ORDERKEY) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"); + + createTable("CREATE TABLE IF NOT EXISTS orders_auto (\n" + + " O_ORDERKEY INTEGER NOT NULL,\n" + + " O_CUSTKEY INTEGER NOT NULL,\n" + + " O_ORDERSTATUS CHAR(1) NOT NULL,\n" + + " O_TOTALPRICE DECIMALV3(15,2) NOT NULL,\n" + + " O_ORDERDATE DATE NOT NULL,\n" + + " O_ORDERPRIORITY CHAR(15) NOT NULL, \n" + + " O_CLERK CHAR(15) NOT NULL, \n" + + " O_SHIPPRIORITY INTEGER NOT NULL,\n" + + " O_COMMENT VARCHAR(79) NOT NULL\n" + + ")\n" + + "DUPLICATE KEY(O_ORDERKEY, O_CUSTKEY)\n" + + "auto partition by range (date_trunc(`O_ORDERDATE`, 'day')) ()\n" + + "DISTRIBUTED BY HASH(O_ORDERKEY) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ")"); + + 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" + + " )"); + // 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"); + } + + // inner join + self join + partition in join condition + valid side + @Test + public void test1() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " inner join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, ImmutableSet.of( + ImmutableList.of("lineitem", "l_shipdate", "true", "true")), + ""); + }); + } + + @Test + public void test100() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(l1.l_shipdate) as col_count\n" + + " from lineitem l1\n" + + " inner join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfo("col_count", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + Assertions.assertTrue(relatedTableInfo.getFailReason().contains( + "partition column use invalid implicit expression")); + Assertions.assertFalse(relatedTableInfo.isPctPossible()); + }); + } + + // inner join + self join + partition in join condition + invalid side + @Test + public void test2() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " inner join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, ImmutableSet.of( + ImmutableList.of("lineitem", "l_shipdate", "true", "true")), + ""); + }); + } + + // inner join + self join + partition not in join condition + valid side + @Test + public void test3() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " inner join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in join invalid side, but is not in join condition"); + }); + } + + // inner join + self join + partition not in join condition + invalid side + @Test + public void test4() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " inner join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in invalid catalog relation to check"); + }); + } + + // inner join + not self join + partition in join condition + valid side + @Test + public void test5() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("orders", "o_orderdate", "true", "true"), ImmutableList.of("lineitem", "l_shipdate", "true", "true")), ""); + }); + } + + @Test + public void test502() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem_auto\n" + + " inner join orders_auto\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem_auto", "l_shipdate", "true", "true"), + ImmutableList.of("orders_auto", "o_orderdate", "true", "true")), ""); + }); + } + + @Test + public void test500() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o.o_orderdate_alias, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join (select date_trunc(o_orderdate, 'day') o_orderdate_alias from orders) o\n" + + " on l_shipdate = o.o_orderdate_alias\n" + + " group by l_shipdate, o.o_orderdate_alias", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true")), ""); + }); + } + + @Test + public void test5000() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o.o_orderdate_alias, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join (select date_trunc(o_orderdate, 'day') o_orderdate_alias from orders) o\n" + + " on l_shipdate = o.o_orderdate_alias\n" + + " group by l_shipdate, o.o_orderdate_alias", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate_alias", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("orders", "o_orderdate", "true", "true"), + ImmutableList.of("lineitem", "l_shipdate", "true", "true")), "day"); + }); + } + + @Test + public void test501() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate_alias, o.o_orderdate_alias, count(l_shipdate_alias) \n" + + " from (select date_trunc(l_shipdate, 'day') l_shipdate_alias from lineitem) l\n" + + " inner join (select date_trunc(o_orderdate, 'day') o_orderdate_alias from orders) o\n" + + " on l_shipdate_alias = o.o_orderdate_alias\n" + + " group by l_shipdate_alias, o.o_orderdate_alias", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate_alias", "month", + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true")), "month"); + }); + } + + // inner join + not self join + partition in join condition + invalid side + @Test + public void test6() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true"), + ImmutableList.of("orders", "o_orderdate", "true", "true")), ""); + }); + } + + @Test + public void test601() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem_auto\n" + + " inner join orders_auto\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem_auto", "l_shipdate", "true", "true"), + ImmutableList.of("orders_auto", "o_orderdate", "true", "true")), ""); + }); + } + + // inner join + not self join + partition not in join condition + valid side + @Test + public void test7() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true")), ""); + }); + } + + // inner join + not self join + partition not in join condition + invalid side + @Test + public void test8() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " inner join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("orders", "o_orderdate", "true", "true")), ""); + }); + } + + + + // left outer join + self join + partition in join condition + valid side + @Test + public void test9() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " left outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "partition column is in join invalid side"); + }); + } + + // left outer join + self join + partition in join condition + invalid side + @Test + public void test10() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " left outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "partition column is in un supported join null generate side"); + }); + } + + // left outer join + self join + partition not in join condition + valid side + @Test + public void test11() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " left outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in invalid catalog relation to check"); + }); + } + + // left outer join + self join + partition not in join condition + invalid side + @Test + public void test12() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " left outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // left outer join + not self join + partition in join condition + valid side + @Test + public void test13() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " left outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true")), ""); + }); + } + + // left outer join + not self join + partition in join condition + invalid side + @Test + public void test14() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " left outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // left outer join + not self join + partition not in join condition + valid side + @Test + public void test15() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " left outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true")), ""); + }); + } + + // left outer join + not self join + partition not in join condition + invalid side + @Test + public void test16() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " left outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + + // right outer join + self join + partition in join condition + valid side + @Test + public void test17() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " right outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // right outer join + self join + partition in join condition + invalid side + @Test + public void test18() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " right outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "partition column is in join invalid side, but is not in join condition"); + }); + } + + // right outer join + self join + partition not in join condition + valid side + @Test + public void test19() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " right outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // right outer join + self join + partition not in join condition + invalid side + @Test + public void test20() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " right outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in join invalid side, but is not in join condition"); + }); + } + + // right outer join + not self join + partition in join condition + invalid side + @Test + public void test21() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " right outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // right outer join + not self join + partition in join condition + valid side + @Test + public void test22() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " right outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("orders", "o_orderdate", "true", "true")), ""); + }); + } + + // right outer join + not self join + partition not in join condition + invalid side + @Test + public void test23() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " right outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // right outer join + not self join + partition not in join condition + valid side + @Test + public void test24() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " right outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, + ImmutableSet.of(ImmutableList.of("orders", "o_orderdate", "true", "true")), ""); + }); + } + + + // full outer join + self join + partition in join condition + valid side + @Test + public void test25() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " full outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + // tmp, wait equal set ignore null ready + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + self join + partition in join condition + invalid side + @Test + public void test26() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " full outer join lineitem l2\n" + + " on l1.l_shipdate = l2.l_shipdate\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + // tmp, wait equal set ignore null ready + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + self join + partition not in join condition + valid side + @Test + public void test27() { + PlanChecker.from(connectContext) + .checkExplain(" select l1.l_shipdate, l2.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " full outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l1.l_shipdate, l2.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + self join + partition not in join condition + invalid side + @Test + public void test28() { + PlanChecker.from(connectContext) + .checkExplain(" select l2.l_shipdate, l1.L_ORDERKEY, count(*)\n" + + " from lineitem l1\n" + + " full outer join lineitem l2\n" + + " on l1.l_orderkey = l2.l_orderkey\n" + + " group by l2.l_shipdate, l1.L_ORDERKEY", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + not self join + partition in join condition + invalid side + @Test + public void test29() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " full outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + not self join + partition in join condition + valid side + @Test + public void test30() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " full outer join orders\n" + + " on l_shipdate = o_orderdate\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + not self join + partition not in join condition + invalid side + @Test + public void test31() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " full outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("l_shipdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // full outer join + not self join + partition not in join condition + valid side + @Test + public void test32() { + PlanChecker.from(connectContext) + .checkExplain(" select l_shipdate, o_orderdate, count(l_shipdate) \n" + + " from lineitem\n" + + " full outer join orders\n" + + " on l_orderkey = o_orderkey\n" + + " group by l_shipdate, o_orderdate", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("o_orderdate", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, + "partition column is in un supported join null generate side"); + }); + } + + // union all input1 is partitioned + input2 is not partitioned but match incremental refresh + @Test + public void test33() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select O_ORDERDATE, O_ORDERDATE_NOT\n" + + "from\n" + + "orders_no_part;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "not union all output pass partition increment check"); + }); + } + + // union all input1 is partitioned + input2 is not partitioned not match incremental refresh + @Test + public void test34() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select date_add(O_ORDERDATE, INTERVAL 1 day) as a, O_ORDERDATE_NOT\n" + + "from\n" + + "orders_no_part", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "union all output doesn't match the partition increment check"); + }); + } + + // union all input1 is partitioned + input2 is partitioned but match incremental refresh + use partition column + @Test + public void test35() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select O_ORDERDATE, O_ORDERDATE_NOT\n" + + "from\n" + + "orders;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true"), + ImmutableList.of("orders", "o_orderdate", "true", "true")), + ""); + }); + } + + + // union all input1 is partitioned + input2 is partitioned not match incremental refresh + use partition column + @Test + public void test36() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select date_add(O_ORDERDATE, INTERVAL 1 day) as a, O_ORDERDATE_NOT\n" + + "from\n" + + "orders;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "union all output doesn't match the partition increment check"); + }); + } + + + // union all input1 is partitioned + input2 is partitioned but match incremental refresh + not use partition column + @Test + public void test37() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select O_ORDERDATE_NOT, O_ORDERDATE\n" + + "from\n" + + "orders;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "not union all output pass partition increment check"); + }); + } + + // union all input1 is partitioned + input2 is partitioned not match incremental refresh + not use partition column + @Test + public void test38() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select date_add(O_ORDERDATE_NOT, INTERVAL 1 day) as a, O_ORDERDATE\n" + + "from\n" + + "orders;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "not union all output pass partition increment check"); + }); + } + + // union all input1 is partitioned + input2 is not partitioned but match incremental refresh with date_trunc + @Test + public void test39() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union all\n" + + "select O_ORDERDATE, O_ORDERDATE_NOT\n" + + "from\n" + + "orders_no_part;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", "month", + rewrittenPlan, nereidsPlanner.getCascadesContext()); + failWith(relatedTableInfo, "not union all output pass partition increment check"); + }); + } + + + // test with cte + @Test + public void test40() { + PlanChecker.from(connectContext) + .checkExplain("with c1 as (\n" + + " select \n" + + " l_shipdate, \n" + + " o_orderdate, \n" + + " count(l_shipdate) as count_s \n" + + " from \n" + + " lineitem \n" + + " inner join orders on l_shipdate = o_orderdate \n" + + " group by \n" + + " l_shipdate, \n" + + " o_orderdate\n" + + "), \n" + + "c2 as (\n" + + " select \n" + + " l_shipdate, \n" + + " o_orderdate, \n" + + " count_s \n" + + " from \n" + + " c1\n" + + "), \n" + + "c3 as (\n" + + " select \n" + + " l_shipdate, \n" + + " count_s \n" + + " from \n" + + " c1\n" + + ") \n" + + "select \n" + + " c2.l_shipdate, \n" + + " c3.count_s \n" + + "from \n" + + " c2 \n" + + " inner join c3 on c2.l_shipdate = c3.l_shipdate;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, ImmutableSet.of( + ImmutableList.of("lineitem", "l_shipdate", "true", "true"), + ImmutableList.of("orders", "o_orderdate", "true", "true")), + null); + }); + } + + + // test with union but not union all + @Test + public void test41() { + PlanChecker.from(connectContext) + .checkExplain("select L_SHIPDATE, L_COMMITDATE\n" + + "from\n" + + "lineitem\n" + + "union\n" + + "select O_ORDERDATE, O_ORDERDATE_NOT\n" + + "from\n" + + "orders;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + RelatedTableInfo relatedTableInfo = + MaterializedViewUtils.getRelatedTableInfos("L_SHIPDATE", null, + rewrittenPlan, nereidsPlanner.getCascadesContext()); + successWith(relatedTableInfo, ImmutableSet.of(ImmutableList.of("lineitem", "l_shipdate", "true", "true"), + ImmutableList.of("orders", "o_orderdate", "true", "true")), + ""); + }); + } + + private static void successWith(RelatedTableInfo relatedTableInfo, + Set> expectTableColumnPairSet, String timeUnit) { + Assertions.assertFalse(relatedTableInfo.getTableColumnInfos().isEmpty()); + Assertions.assertTrue(relatedTableInfo.isPctPossible()); + + Set> relatedTableColumnPairs = new HashSet<>(); + List tableColumnInfos = relatedTableInfo.getTableColumnInfos(); + boolean anyFoundDateTrunc = false; + for (RelatedTableColumnInfo info : tableColumnInfos) { + Optional partitionExpression = info.getPartitionExpression(); + if (StringUtils.isNotEmpty(timeUnit) && !partitionExpression.isPresent()) { + Assertions.fail("excepted time unit in partition expression but not"); + } + if (StringUtils.isNotEmpty(timeUnit) && partitionExpression.isPresent()) { + List dateTruncs = partitionExpression.get().collectToList(DateTrunc.class::isInstance); + anyFoundDateTrunc = anyFoundDateTrunc + || (dateTruncs.size() == 1 + && Objects.equals("'" + timeUnit + "'", dateTruncs.get(0).getArgument(1).toString().toLowerCase())); + } + try { + relatedTableColumnPairs.add( + ImmutableList.of(info.getTableInfo().getTableName(), info.getColumnStr().toLowerCase(), + String.valueOf(info.isReachRelationCheck()), String.valueOf(info.isFromTablePartitionColumn()))); + } catch (Exception exception) { + Assertions.fail("excepted table and column in related table column info but not"); + } + } + if (StringUtils.isNotEmpty(timeUnit)) { + Assertions.assertTrue(anyFoundDateTrunc); + } + Assertions.assertEquals(expectTableColumnPairSet, relatedTableColumnPairs); + } + + private static void failWith(RelatedTableInfo relatedTableInfo, + String failInfo) { + Assertions.assertFalse(relatedTableInfo.isPctPossible()); + Assertions.assertTrue(relatedTableInfo.getFailReason().contains(failInfo)); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java index 8e997062f50bfd..c23ff24c868d89 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java @@ -718,11 +718,11 @@ public void onInvokeRule(RuleType ruleType) { } public PlanChecker checkExplain(String sql, Consumer consumer) { - LogicalPlan parsed = new NereidsParser().parseSingle(sql); StatementContext statementContext = new StatementContext(connectContext, new OriginStatement(sql, 0)); - NereidsPlanner nereidsPlanner = new NereidsPlanner( - statementContext); connectContext.setStatementContext(statementContext); + + LogicalPlan parsed = new NereidsParser().parseSingle(sql); + NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); LogicalPlanAdapter adapter = LogicalPlanAdapter.of(parsed); adapter.setIsExplain(new ExplainOptions(ExplainLevel.ALL_PLAN, false)); SessionVariable sessionVariable = connectContext.getSessionVariable(); @@ -745,10 +745,11 @@ public PlanChecker checkExplain(String sql, Consumer consumer) { } public PlanChecker checkPlannerResult(String sql, Consumer consumer) { - LogicalPlan parsed = new NereidsParser().parseSingle(sql); StatementContext statementContext = new StatementContext(connectContext, new OriginStatement(sql, 0)); - NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); connectContext.setStatementContext(statementContext); + + LogicalPlan parsed = new NereidsParser().parseSingle(sql); + NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext); SessionVariable sessionVariable = connectContext.getSessionVariable(); try { nereidsPlanner.plan(LogicalPlanAdapter.of(parsed)); diff --git a/regression-test/data/mtmv_p0/test_multi_pct_list_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_list_mtmv.out new file mode 100644 index 00000000000000..f2274535952178 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_list_mtmv.out @@ -0,0 +1,24 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2 2 3 2 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2 2 3 2 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2 2 3 2 +2 2 6 2 + +-- !refresh_mode_t3 -- +COMPLETE + +-- !4 -- +2 2 3 2 +2 2 6 2 + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_mtmv.out new file mode 100644 index 00000000000000..040c49df8cee48 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_mtmv.out @@ -0,0 +1,43 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + +-- !refresh_mode_t3 -- +COMPLETE + +-- !4 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + +-- !partitions_1 -- +p_20170101_20170201 [] +p_20170201_20170301 [test_multi_pct_mtmv_table1] +p_20170301_20170401 [] + +-- !partitions_2 -- +p_20170101_20170201 [] +p_20170201_20170301 [] +p_20170301_20170401 [] + +-- !partitions_3 -- +p_20170101_20170201 [] +p_20170201_20170301 [] +p_20170301_20170401 [test_multi_pct_mtmv_table2] + +-- !partitions_4 -- +p_20170101_20170201 [] +p_20170201_20170301 [] + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_nested_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_nested_mtmv.out new file mode 100644 index 00000000000000..e4acea3b37a473 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_nested_mtmv.out @@ -0,0 +1,24 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + +-- !refresh_mode_t3 -- +COMPLETE + +-- !4 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_rollup_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_rollup_mtmv.out new file mode 100644 index 00000000000000..b02f0c27562af9 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_rollup_mtmv.out @@ -0,0 +1,28 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2037-01-01 6 +2038-01-01 2 +2038-02-01 3 +2038-03-01 5 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2037-01-01 6 +2038-01-01 2 +2038-01-01 7 +2038-02-01 3 +2038-03-01 5 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2037-01-01 6 +2037-01-01 8 +2038-01-01 2 +2038-01-01 7 +2038-02-01 3 +2038-03-01 5 + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_union_list_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_union_list_mtmv.out new file mode 100644 index 00000000000000..66057133b5c012 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_union_list_mtmv.out @@ -0,0 +1,28 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +1 1 +2 2 +2 3 +3 4 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +1 1 +1 5 +2 2 +2 3 +3 4 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +1 1 +1 5 +2 2 +2 3 +2 6 +3 4 + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_union_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_union_mtmv.out new file mode 100644 index 00000000000000..a7a9589ed746dc --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_union_mtmv.out @@ -0,0 +1,28 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2017-01-01 1 +2017-02-01 2 +2017-02-01 3 +2017-03-01 4 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2017-01-01 1 +2017-01-01 5 +2017-02-01 2 +2017-02-01 3 +2017-03-01 4 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2017-01-01 1 +2017-01-01 5 +2017-02-01 2 +2017-02-01 3 +2017-02-01 6 +2017-03-01 4 + diff --git a/regression-test/data/mtmv_p0/test_multi_pct_view_mtmv.out b/regression-test/data/mtmv_p0/test_multi_pct_view_mtmv.out new file mode 100644 index 00000000000000..e4acea3b37a473 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_multi_pct_view_mtmv.out @@ -0,0 +1,24 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !1 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t1 -- +PARTIAL + +-- !2 -- +2017-02-01 2 3 2 + +-- !refresh_mode_t2 -- +PARTIAL + +-- !3 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + +-- !refresh_mode_t3 -- +COMPLETE + +-- !4 -- +2017-02-01 2 3 2 +2017-02-01 2 6 2 + diff --git a/regression-test/data/mtmv_p0/test_null_multi_pct_mtmv.out b/regression-test/data/mtmv_p0/test_null_multi_pct_mtmv.out new file mode 100644 index 00000000000000..1cb40b550ba711 --- /dev/null +++ b/regression-test/data/mtmv_p0/test_null_multi_pct_mtmv.out @@ -0,0 +1,27 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !list_null -- +3 \N +4 \N + +-- !list_1 -- +1 1 + +-- !list_2 -- +2 2 + +-- !range_p_32768_10 -- +1 \N +3 \N + +-- !range_p_10_20 -- +2 15 +4 16 + +-- !range_p_00000101_20201111 -- +1 \N +3 \N + +-- !range_p_20201111_20211111 -- +2 2021-01-01 +4 2021-01-01 + diff --git a/regression-test/data/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.out b/regression-test/data/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.out new file mode 100644 index 00000000000000..a059dcd97a178a --- /dev/null +++ b/regression-test/data/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.out @@ -0,0 +1,10 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !refresh_init -- +2017-02-01 2 3 + +-- !mtmv_sync -- +true + +-- !refresh_sc -- +2017-02-01 2 3 + diff --git a/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.out b/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.out new file mode 100644 index 00000000000000..5036ccab9cf405 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.out @@ -0,0 +1,41 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query_1_0_before -- +2023-10-17 2023-10-17 2 3 12.00 +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 3.00 +2023-10-22 2023-10-22 2 3 24.00 + +-- !query_1_0_after -- +2023-10-17 2023-10-17 2 3 12.00 +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 3.00 +2023-10-22 2023-10-22 2 3 24.00 + +-- !query_2_0_before -- +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 19.00 +2023-10-22 2023-10-22 2 3 24.00 + +-- !query_2_0_after -- +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 19.00 +2023-10-22 2023-10-22 2 3 24.00 + +-- !query_3_0_before -- +2023-10-17 2023-10-17 2 3 21.00 +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 3.00 +2023-10-22 2023-10-22 2 3 24.00 + +-- !query_3_0_after -- +2023-10-17 2023-10-17 2 3 21.00 +2023-10-18 2023-10-18 2 3 18.00 +2023-10-19 2023-10-19 2 3 24.00 +2023-10-20 2023-10-20 2 3 3.00 +2023-10-22 2023-10-22 2 3 24.00 + diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_bad_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_bad_mtmv.groovy new file mode 100644 index 00000000000000..571045400ff95d --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_bad_mtmv.groovy @@ -0,0 +1,377 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_bad_mtmv","mtmv") { + String suiteName = "test_multi_pct_bad_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String tableName3 = "${suiteName}_table3" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-01-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + String mvSql = "SELECT t1.k1,t1.k2,t2.k2 as k3 from ${tableName1} t1 inner join ${tableName2} t2 on t1.k1=t2.k1;"; + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + exception "intersected" + } + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")), + PARTITION `p201703` VALUES [("2017-03-01"), ("2017-04-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName3} + ( + k3 INT not null, + k4 INT not null + ) + DISTRIBUTED BY HASH(k3) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1} + union all + select * from ${tableName2} + union all + select * from ${tableName3} + """ + exception "suitable" + } + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES LESS THAN ("2017-01-01"), + PARTITION `p201702` VALUES LESS THAN ("2017-02-01"), + PARTITION `pmax` VALUES LESS THAN (maxvalue) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES LESS THAN ("2017-01-01"), + PARTITION `p201703` VALUES LESS THAN ("2017-03-01"), + PARTITION `pmax` VALUES LESS THAN (maxvalue) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1} + union all + select * from ${tableName2} + """ + exception "intersected" + } + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p1` VALUES IN ("1","2"), + PARTITION `p2` VALUES IN ("3") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p2` VALUES IN ("2"), + PARTITION `p3` VALUES IN ("4") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1} + union all + select * from ${tableName2} + """ + exception "repeat" + } + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p2` VALUES IN ("2017-01-01"), + PARTITION `p3` VALUES IN ("2017-02-01") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1} + union all + select * from ${tableName2} + """ + exception "same" + } + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p1` VALUES IN ("1"), + PARTITION `p2` VALUES IN ("2") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 bigint not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p1` VALUES IN ("1"), + PARTITION `p2` VALUES IN ("2") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + test { + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1} + union all + select * from ${tableName2} + """ + exception "suitable" + } +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_list_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_list_mtmv.groovy new file mode 100644 index 00000000000000..47d8afaa9173a3 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_list_mtmv.groovy @@ -0,0 +1,152 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_list_mtmv","mtmv") { + String suiteName = "test_multi_pct_list_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String tableName3 = "${suiteName}_table3" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p1` VALUES IN ("1"), + PARTITION `p2` VALUES IN ("2") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values(1,1); + insert into ${tableName1} values(2,2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p2` VALUES IN ("2"), + PARTITION `p3` VALUES IN ("3") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values(2,3); + insert into ${tableName2} values(3,4); + """ + + sql """ + CREATE TABLE ${tableName3} + ( + k3 INT not null, + k4 INT not null + ) + DISTRIBUTED BY HASH(k3) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into ${tableName3} values(1,1); + insert into ${tableName3} values(2,2); + insert into ${tableName3} values(3,3); + insert into ${tableName3} values(4,4); + """ + String mvSql = "SELECT t1.k1,t1.k2,t2.k2 as k3,t3.k4 from ${tableName1} t1 inner join ${tableName2} t2 on t1.k1=t2.k1 left join ${tableName3} t3 on t1.k2=t3.k3;"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_1")) + assertTrue(showPartitionsResult.toString().contains("p_2")) + assertTrue(showPartitionsResult.toString().contains("p_3")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values(1,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values(2,6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName3} values(5,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t3 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_4 "SELECT * FROM ${mvName}" + + mv_rewrite_success_without_check_chosen(mvSql, mvName) +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_mtmv.groovy new file mode 100644 index 00000000000000..17a39441a08a26 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_mtmv.groovy @@ -0,0 +1,175 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_mtmv","mtmv") { + String suiteName = "test_multi_pct_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String tableName3 = "${suiteName}_table3" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values("2017-01-01",1); + insert into ${tableName1} values("2017-02-01",2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")), + PARTITION `p201703` VALUES [("2017-03-01"), ("2017-04-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values("2017-02-01",3); + insert into ${tableName2} values("2017-03-01",4); + """ + + sql """ + CREATE TABLE ${tableName3} + ( + k3 INT not null, + k4 INT not null + ) + DISTRIBUTED BY HASH(k3) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into ${tableName3} values(1,1); + insert into ${tableName3} values(2,2); + insert into ${tableName3} values(3,3); + insert into ${tableName3} values(4,4); + """ + String mvSql = "SELECT t1.k1,t1.k2,t2.k2 as k3,t3.k4 from ${tableName1} t1 inner join ${tableName2} t2 on t1.k1=t2.k1 left join ${tableName3} t3 on t1.k2=t3.k3;"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20170101_20170201")) + assertTrue(showPartitionsResult.toString().contains("p_20170201_20170301")) + assertTrue(showPartitionsResult.toString().contains("p_20170301_20170401")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values("2017-01-01",5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values("2017-02-01",6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName3} values(5,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t3 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_4 "SELECT * FROM ${mvName}" + + mv_rewrite_success_without_check_chosen(mvSql, mvName) + + sql """ + alter table ${tableName1} drop partition p201702; + """ + + order_qt_partitions_1 "select PartitionName,UnsyncTables from partitions('catalog'='internal','database'='${dbName}','table'='${mvName}') order by PartitionId desc;" + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_partitions_2 "select PartitionName,UnsyncTables from partitions('catalog'='internal','database'='${dbName}','table'='${mvName}') order by PartitionId desc;" + + sql """ + alter table ${tableName2} drop partition p201703; + """ + order_qt_partitions_3 "select PartitionName,UnsyncTables from partitions('catalog'='internal','database'='${dbName}','table'='${mvName}') order by PartitionId desc;" + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_partitions_4 "select PartitionName,UnsyncTables from partitions('catalog'='internal','database'='${dbName}','table'='${mvName}') order by PartitionId desc;" +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_nested_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_nested_mtmv.groovy new file mode 100644 index 00000000000000..e483a661801722 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_nested_mtmv.groovy @@ -0,0 +1,204 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_nested_mtmv","mtmv") { + String suiteName = "test_multi_pct_nested_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String tableName3 = "${suiteName}_table3" + String mvName1 = "${suiteName}_mv1" + String mvName2 = "${suiteName}_mv2" + String mvName3 = "${suiteName}_mv3" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop materialized view if exists ${mvName1};""" + sql """drop materialized view if exists ${mvName2};""" + sql """drop materialized view if exists ${mvName3};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values("2017-01-01",1); + insert into ${tableName1} values("2017-02-01",2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")), + PARTITION `p201703` VALUES [("2017-03-01"), ("2017-04-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values("2017-02-01",3); + insert into ${tableName2} values("2017-03-01",4); + """ + + sql """ + CREATE TABLE ${tableName3} + ( + k3 INT not null, + k4 INT not null + ) + DISTRIBUTED BY HASH(k3) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into ${tableName3} values(1,1); + insert into ${tableName3} values(2,2); + insert into ${tableName3} values(3,3); + insert into ${tableName3} values(4,4); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName1} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName1}; + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName1} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName1) + + sql """ + CREATE MATERIALIZED VIEW ${mvName2} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + select * from ${tableName2}; + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName2} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName2) + + String mvSql = "SELECT t1.k1,t1.k2,t2.k2 as k3,t3.k4 from ${mvName1} t1 inner join ${mvName2} t2 on t1.k1=t2.k1 left join ${tableName3} t3 on t1.k2=t3.k3;"; + sql """ + CREATE MATERIALIZED VIEW ${mvName3} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName3}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20170101_20170201")) + assertTrue(showPartitionsResult.toString().contains("p_20170201_20170301")) + assertTrue(showPartitionsResult.toString().contains("p_20170301_20170401")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName3} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName3) + + order_qt_1 "SELECT * FROM ${mvName3}" + + sql """ + insert into ${tableName1} values("2017-01-01",5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName1} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName1) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName3} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName3) + + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName3}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName3}" + + sql """ + insert into ${tableName2} values("2017-02-01",6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName2} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName2) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName3} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName3) + + + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName3}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName3}" + + sql """ + insert into ${tableName3} values(5,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName3} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName3) + order_qt_refresh_mode_t3 "select RefreshMode from tasks('type'='mv') where MvName='${mvName3}' order by CreateTime desc limit 1" + order_qt_4 "SELECT * FROM ${mvName3}" + + mv_rewrite_success_without_check_chosen(mvSql, mvName3) +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_rollup_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_rollup_mtmv.groovy new file mode 100644 index 00000000000000..7c20343c35709e --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_rollup_mtmv.groovy @@ -0,0 +1,124 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_rollup_mtmv","mtmv") { + String suiteName = "test_multi_pct_rollup_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p203801` VALUES [("2038-01-01"), ("2038-02-01")), + PARTITION `p203802` VALUES [("2038-02-01"), ("2038-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values("2017-01-01",1); + insert into ${tableName1} values("2038-01-01",2); + insert into ${tableName1} values("2038-02-01",3); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201601` VALUES [("2016-01-01"), ("2016-02-01")), + PARTITION `p203802` VALUES [("2038-03-01"), ("2038-04-01")), + PARTITION `p203803` VALUES [("2037-01-01"), ("2037-02-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values("2016-01-01",4); + insert into ${tableName2} values("2038-03-01",5); + insert into ${tableName2} values("2037-01-01",6); + """ + + String mvSql = "SELECT * from ${tableName1} union all SELECT * from ${tableName2};"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(date_trunc(`k1`, 'year')) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'partition_sync_limit'='2', + 'partition_sync_time_unit'='YEAR', + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20370101_20380101")) + assertTrue(showPartitionsResult.toString().contains("p_20380101_20390101")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values("2038-01-01",7); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values("2037-01-01",8); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_union_list_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_union_list_mtmv.groovy new file mode 100644 index 00000000000000..7f10b881246418 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_union_list_mtmv.groovy @@ -0,0 +1,118 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_union_list_mtmv","mtmv") { + String suiteName = "test_multi_pct_union_list_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p1` VALUES IN ("1"), + PARTITION `p2` VALUES IN ("2") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values(1,1); + insert into ${tableName1} values(2,2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 INT not null, + k2 INT not null + ) + PARTITION BY LIST(`k1`) + ( + PARTITION `p2` VALUES IN ("2"), + PARTITION `p3` VALUES IN ("3") + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values(2,3); + insert into ${tableName2} values(3,4); + """ + String mvSql = "SELECT * from ${tableName1} union all SELECT * from ${tableName2};"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_1")) + assertTrue(showPartitionsResult.toString().contains("p_2")) + assertTrue(showPartitionsResult.toString().contains("p_3")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values(1,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values(2,6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_union_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_union_mtmv.groovy new file mode 100644 index 00000000000000..cf5c92245c2034 --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_union_mtmv.groovy @@ -0,0 +1,119 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_union_mtmv","mtmv") { + String suiteName = "test_multi_pct_union_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values("2017-01-01",1); + insert into ${tableName1} values("2017-02-01",2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")), + PARTITION `p201703` VALUES [("2017-03-01"), ("2017-04-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values("2017-02-01",3); + insert into ${tableName2} values("2017-03-01",4); + """ + + String mvSql = "SELECT * from ${tableName1} union all SELECT * from ${tableName2};"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20170101_20170201")) + assertTrue(showPartitionsResult.toString().contains("p_20170201_20170301")) + assertTrue(showPartitionsResult.toString().contains("p_20170301_20170401")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values("2017-01-01",5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values("2017-02-01",6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" +} diff --git a/regression-test/suites/mtmv_p0/test_multi_pct_view_mtmv.groovy b/regression-test/suites/mtmv_p0/test_multi_pct_view_mtmv.groovy new file mode 100644 index 00000000000000..e31231af4b562a --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_multi_pct_view_mtmv.groovy @@ -0,0 +1,159 @@ +// 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. + +import org.junit.Assert; + +suite("test_multi_pct_view_mtmv","mtmv") { + String suiteName = "test_multi_pct_view_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String tableName3 = "${suiteName}_table3" + String viewName = "${suiteName}_view" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop table if exists `${tableName3}`""" + sql """drop view if exists `${viewName}`""" + sql """drop materialized view if exists ${mvName};""" + + sql """ + CREATE TABLE ${tableName1} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201701` VALUES [("2017-01-01"), ("2017-02-01")), + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName1} values("2017-01-01",1); + insert into ${tableName1} values("2017-02-01",2); + """ + + sql """ + CREATE TABLE ${tableName2} + ( + k1 DATE not null, + k2 INT not null + ) + PARTITION BY RANGE(`k1`) + ( + PARTITION `p201702` VALUES [("2017-02-01"), ("2017-03-01")), + PARTITION `p201703` VALUES [("2017-03-01"), ("2017-04-01")) + ) + DISTRIBUTED BY HASH(k2) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + sql """ + insert into ${tableName2} values("2017-02-01",3); + insert into ${tableName2} values("2017-03-01",4); + """ + + sql """ + CREATE TABLE ${tableName3} + ( + k3 INT not null, + k4 INT not null + ) + DISTRIBUTED BY HASH(k3) BUCKETS 2 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + insert into ${tableName3} values(1,1); + insert into ${tableName3} values(2,2); + insert into ${tableName3} values(3,3); + insert into ${tableName3} values(4,4); + """ + + sql""" + create view ${viewName} as SELECT t1.k1,t1.k2,t2.k2 as k3 from ${tableName1} t1 inner join ${tableName2} t2 on t1.k1=t2.k1; + """ + + String mvSql = "select v1.k1,v1.k2,v1.k3,t3.k4 from ${viewName} v1 left join ${tableName3} t3 on v1.k2=t3.k3;"; + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(k1) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ( + 'replication_num' = '1' + ) + AS + ${mvSql} + """ + + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_20170101_20170201")) + assertTrue(showPartitionsResult.toString().contains("p_20170201_20170301")) + assertTrue(showPartitionsResult.toString().contains("p_20170301_20170401")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_1 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName1} values("2017-01-01",5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t1 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_2 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName2} values("2017-02-01",6); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t2 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_3 "SELECT * FROM ${mvName}" + + sql """ + insert into ${tableName3} values(5,5); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_mode_t3 "select RefreshMode from tasks('type'='mv') where MvName='${mvName}' order by CreateTime desc limit 1" + order_qt_4 "SELECT * FROM ${mvName}" + + mv_rewrite_success_without_check_chosen(mvSql, mvName) +} diff --git a/regression-test/suites/mtmv_p0/test_null_multi_pct_mtmv.groovy b/regression-test/suites/mtmv_p0/test_null_multi_pct_mtmv.groovy new file mode 100644 index 00000000000000..390d9c05aad58a --- /dev/null +++ b/regression-test/suites/mtmv_p0/test_null_multi_pct_mtmv.groovy @@ -0,0 +1,231 @@ +// 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. + +import org.junit.Assert; + +suite("test_null_multi_pct_mtmv", "mtmv") { +String suiteName = "test_multi_pct_bad_mtmv" + String dbName = context.config.getDbNameByFile(context.file) + String tableName1 = "${suiteName}_table1" + String tableName2 = "${suiteName}_table2" + String mvName = "${suiteName}_mv" + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + // list table + sql """ + CREATE TABLE `${tableName1}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY list(`num`) + ( + PARTITION p_1 VALUES IN (1), + PARTITION p_null VALUES IN (null) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName1} values(1,1),(3,null); + """ + + sql """ + CREATE TABLE `${tableName2}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY list(`num`) + ( + PARTITION p_2 VALUES IN (2), + PARTITION p_null VALUES IN (null) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName2} values(2,2),(4,null); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`num`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName1} + union all + SELECT * FROM ${tableName2}; + """ + def showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_1")) + assertTrue(showPartitionsResult.toString().contains("p_2")) + assertTrue(showPartitionsResult.toString().contains("p_NULL")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_list_null "SELECT * FROM ${mvName} partitions(p_NULL) order by user_id,num" + order_qt_list_1 "SELECT * FROM ${mvName} partitions(p_1) order by user_id,num" + order_qt_list_2 "SELECT * FROM ${mvName} partitions(p_2) order by user_id,num" + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + + // range table + sql """ + CREATE TABLE `${tableName1}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY range(`num`) + ( + PARTITION p_10 VALUES LESS THAN (10), + PARTITION p_20 VALUES LESS THAN (20) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName1} values(1,null),(2,15); + """ + + // range table + sql """ + CREATE TABLE `${tableName2}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `num` SMALLINT COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `num`) + COMMENT 'OLAP' + PARTITION BY range(`num`) + ( + PARTITION p_10 VALUES LESS THAN (10), + PARTITION p_20 VALUES LESS THAN (20) + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName2} values(3,null),(4,16); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`num`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName1} + union all + SELECT * FROM ${tableName2}; + """ + showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_32768_10")) + assertTrue(showPartitionsResult.toString().contains("p_10_20")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_range_p_32768_10 "SELECT * FROM ${mvName} partitions(p_32768_10) order by user_id,num" + order_qt_range_p_10_20 "SELECT * FROM ${mvName} partitions(p_10_20) order by user_id,num" + + sql """drop table if exists `${tableName1}`""" + sql """drop table if exists `${tableName2}`""" + sql """drop materialized view if exists ${mvName};""" + + // range date table + sql """ + CREATE TABLE `${tableName1}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `create_date` DATE + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `create_date`) + COMMENT 'OLAP' + PARTITION BY range(`create_date`) + ( + PARTITION p_10 VALUES LESS THAN ("2020-11-11"), + PARTITION p_20 VALUES LESS THAN ("2021-11-11") + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName1} values(1,null),(2,"2021-01-01"); + """ + + sql """ + CREATE TABLE `${tableName2}` ( + `user_id` LARGEINT NOT NULL COMMENT '\"用户id\"', + `create_date` DATE + ) ENGINE=OLAP + DUPLICATE KEY(`user_id`, `create_date`) + COMMENT 'OLAP' + PARTITION BY range(`create_date`) + ( + PARTITION p_10 VALUES LESS THAN ("2020-11-11"), + PARTITION p_20 VALUES LESS THAN ("2021-11-11") + ) + DISTRIBUTED BY HASH(`user_id`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName2} values(3,null),(4,"2021-01-01"); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + BUILD DEFERRED REFRESH AUTO ON MANUAL + partition by(`create_date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT * FROM ${tableName1} + union all + SELECT * FROM ${tableName2}; + """ + showPartitionsResult = sql """show partitions from ${mvName}""" + logger.info("showPartitionsResult: " + showPartitionsResult.toString()) + assertTrue(showPartitionsResult.toString().contains("p_00000101_20201111")) + assertTrue(showPartitionsResult.toString().contains("p_20201111_20211111")) + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} AUTO + """ + waitingMTMVTaskFinishedByMvName(mvName) + + order_qt_range_p_00000101_20201111 "SELECT * FROM ${mvName} partitions(p_00000101_20201111) order by user_id,create_date" + order_qt_range_p_20201111_20211111 "SELECT * FROM ${mvName} partitions(p_20201111_20211111) order by user_id,create_date" + +} diff --git a/regression-test/suites/mtmv_up_down_multi_pct_p0/load.groovy b/regression-test/suites/mtmv_up_down_multi_pct_p0/load.groovy new file mode 100644 index 00000000000000..b22003977d15cb --- /dev/null +++ b/regression-test/suites/mtmv_up_down_multi_pct_p0/load.groovy @@ -0,0 +1,76 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_upgrade_downgrade_prepare_multi_pct_mtmv","p0,mtmv,restart_fe") { + String suiteName = "mtmv_up_down_multi_pct" + String mvName = "${suiteName}_mtmv" + String tableName = "${suiteName}_table" + String tableName2 = "${suiteName}_table2" + + sql """drop materialized view if exists ${mvName};""" + sql """drop table if exists `${tableName}`""" + sql """drop table if exists `${tableName2}`""" + + sql """ + CREATE TABLE `${tableName}` ( + `date` DATE NOT NULL COMMENT '\"数据灌入日期时间\"', + `num` SMALLINT NOT NULL COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`date`, `num`) + COMMENT 'OLAP' + PARTITION BY RANGE(`date`) + ( + PARTITION p201701 VALUES [('2017-01-01'), ('2017-02-01')), + PARTITION p201702 VALUES [('2017-02-01'), ('2017-03-01')) + ) + DISTRIBUTED BY HASH(`date`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values("2017-01-01",1),("2017-02-01",2); + """ + + sql """ + CREATE TABLE `${tableName2}` ( + `date` DATE NOT NULL COMMENT '\"数据灌入日期时间\"', + `num` SMALLINT NOT NULL COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`date`, `num`) + COMMENT 'OLAP' + PARTITION BY RANGE(`date`) + ( + PARTITION p201702 VALUES [('2017-02-01'), ('2017-03-01')), + PARTITION p201703_all VALUES [('2017-03-01'), ('2017-04-01')) + ) + DISTRIBUTED BY HASH(`date`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName2} values("2017-02-01",3),("2017-03-01",4); + """ + + sql """ + CREATE MATERIALIZED VIEW ${mvName} + REFRESH AUTO ON MANUAL + partition by(`date`) + DISTRIBUTED BY RANDOM BUCKETS 2 + PROPERTIES ('replication_num' = '1') + AS + SELECT a.*,b.num as num2 FROM ${tableName} a inner join ${tableName2} b on a.`date`= b.`date`; + """ + waitingMTMVTaskFinishedByMvName(mvName) +} diff --git a/regression-test/suites/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.groovy b/regression-test/suites/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.groovy new file mode 100644 index 00000000000000..47ea768f9e75e6 --- /dev/null +++ b/regression-test/suites/mtmv_up_down_multi_pct_p0/test_upgrade_downgrade_multi_pct_mtmv.groovy @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("test_upgrade_downgrade_multi_pct_mtmv","p0,mtmv,restart_fe") { + String suiteName = "mtmv_up_down_multi_pct" + String dbName = context.config.getDbNameByFile(context.file) + String mvName = "${suiteName}_mtmv" + String tableName = "${suiteName}_table" + String tableName2 = "${suiteName}_table2" + // test data is normal + order_qt_refresh_init "SELECT * FROM ${mvName}" + // test is sync + order_qt_mtmv_sync "select SyncWithBaseTables from mv_infos('database'='${dbName}') where Name='${mvName}'" + sql """ + REFRESH MATERIALIZED VIEW ${mvName} complete + """ + // test can refresh success + waitingMTMVTaskFinishedByMvName(mvName) + + // test schema change + sql """drop table if exists `${tableName}`""" + sql """ + CREATE TABLE `${tableName}` ( + `date` DATE NOT NULL COMMENT '\"数据灌入日期时间\"', + `num` SMALLINT NOT NULL COMMENT '\"数量\"' + ) ENGINE=OLAP + DUPLICATE KEY(`date`, `num`) + COMMENT 'OLAP' + PARTITION BY RANGE(`date`) + ( + PARTITION p201701 VALUES [('2017-01-01'), ('2017-02-01')), + PARTITION p201702 VALUES [('2017-02-01'), ('2017-03-01')) + ) + DISTRIBUTED BY HASH(`date`) BUCKETS 2 + PROPERTIES ('replication_num' = '1') ; + """ + sql """ + insert into ${tableName} values("2017-01-01",1),("2017-02-01",2); + """ + + sql """ + REFRESH MATERIALIZED VIEW ${mvName} auto + """ + // test can refresh success + waitingMTMVTaskFinishedByMvName(mvName) + order_qt_refresh_sc "SELECT * FROM ${mvName}" +} + diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_list_str_increment_create.groovy index afaa1a39818bbb..0e24889c007f0a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_list_str_increment_create.groovy @@ -345,12 +345,12 @@ suite("cross_join_list_str_increment_create") { def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -375,9 +375,9 @@ suite("cross_join_list_str_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) // create mv base on left table with no partition col diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy index f4fc3fc91306e4..90c8a9447b2118 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_date_increment_create.groovy @@ -336,12 +336,12 @@ suite("cross_join_range_date_increment_create") { def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_13] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -366,9 +366,9 @@ suite("cross_join_range_date_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_13] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_number_increment_create.groovy index f727152813a67f..0bf234131836a2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/cross_join_range_number_increment_create.groovy @@ -347,10 +347,11 @@ suite("cross_join_range_number_increment_create") { mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -375,9 +376,9 @@ suite("cross_join_range_number_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_list_str_increment_create.groovy index a51eccb2020097..789959e4ccccfd 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_list_str_increment_create.groovy @@ -147,6 +147,7 @@ suite("inner_join_list_str_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "PARTIAL") @@ -163,6 +164,7 @@ suite("inner_join_list_str_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -178,6 +180,7 @@ suite("inner_join_list_str_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -361,12 +364,13 @@ suite("inner_join_list_str_increment_create") { def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_7, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -380,7 +384,7 @@ suite("inner_join_list_str_increment_create") { // create mv base on right table with partition col sql_error_list = [mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_18] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_11, mv_sql_17] + sql_complete_list = [mv_sql_1, mv_sql_3] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col_right, primary_tb_change, is_complete_change_right) @@ -393,8 +397,8 @@ suite("inner_join_list_str_increment_create") { // change right table data // create mv base on left table with partition col sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_increment_list = [mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_complete_list = [mv_sql_1, mv_sql_3] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy index 923000a976123d..36288e1d7f7fb6 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_date_increment_create.groovy @@ -138,6 +138,7 @@ suite("inner_join_range_date_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "PARTIAL") @@ -154,6 +155,7 @@ suite("inner_join_range_date_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -169,6 +171,7 @@ suite("inner_join_range_date_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -354,10 +357,11 @@ suite("inner_join_range_date_increment_create") { mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_13] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -369,8 +373,8 @@ suite("inner_join_range_date_increment_create") { // create mv base on right table with partition col sql_error_list = [mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_17, mv_sql_18] - sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_10, mv_sql_16] + sql_increment_list = [mv_sql_1, mv_sql_3] + sql_complete_list = [] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col_right, primary_tb_change, is_complete_change_right) @@ -382,9 +386,9 @@ suite("inner_join_range_date_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_13] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_increment_list = [mv_sql_1, mv_sql_3] + sql_complete_list = [mv_sql_4, mv_sql_6, mv_sql_7] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_number_increment_create.groovy index 6804be26b0545a..89be330e2b0ad1 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/inner_join_range_number_increment_create.groovy @@ -148,6 +148,7 @@ suite("inner_join_range_number_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "PARTIAL") @@ -164,6 +165,7 @@ suite("inner_join_range_number_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -179,6 +181,7 @@ suite("inner_join_range_number_increment_create") { def refresh_info = sql """select JobName, Status, RefreshMode, NeedRefreshPartitions, CompletedPartitions, Progress from tasks("type"="mv") where JobName="${cur_job_name}" order by CreateTime desc limit 1;""" + logger.info("refresh_info: " + refresh_info) assert (refresh_info[0][0] == cur_job_name) assert (refresh_info[0][1] == "SUCCESS") assert (refresh_info[0][2] == "COMPLETE") @@ -364,10 +367,11 @@ suite("inner_join_range_number_increment_create") { mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -380,7 +384,7 @@ suite("inner_join_range_number_increment_create") { // create mv base on right table with partition col sql_error_list = [mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_12, mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_18] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_11, mv_sql_17] + sql_complete_list = [mv_sql_1, mv_sql_3] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col_right, primary_tb_change, is_complete_change_right) @@ -393,8 +397,8 @@ suite("inner_join_range_number_increment_create") { // change right table data // create mv base on left table with partition col sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_increment_list = [mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_14] + sql_complete_list = [mv_sql_1, mv_sql_3] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_list_str_increment_create.groovy index 62117c298b8fa2..aeefdb8ad1ddcd 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_list_str_increment_create.groovy @@ -302,12 +302,12 @@ suite("left_anti_join_list_str_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -328,9 +328,9 @@ suite("left_anti_join_list_str_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy index c6e1b907193477..f443406831ce71 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_date_increment_create.groovy @@ -298,7 +298,7 @@ suite("left_anti_join_range_date_increment_create") { // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -319,9 +319,9 @@ suite("left_anti_join_range_date_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_10] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_number_increment_create.groovy index 9dbcd4bd92d7c7..9ab74e04f9e881 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_anti_join_range_number_increment_create.groovy @@ -301,12 +301,13 @@ suite("left_anti_join_range_number_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -327,9 +328,9 @@ suite("left_anti_join_range_number_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_list_str_increment_create.groovy index 19b17201d29fc8..4fc907f7ed7a86 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_list_str_increment_create.groovy @@ -302,12 +302,12 @@ suite("left_join_list_str_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -328,9 +328,9 @@ suite("left_join_list_str_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy index 621bc6fa032dcf..51ea3c0f621555 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_date_increment_create.groovy @@ -293,12 +293,12 @@ suite("left_join_range_date_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_10] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -319,9 +319,9 @@ suite("left_join_range_date_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_10] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_number_increment_create.groovy index ab149544ad9dfe..91360b733e10db 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_join_range_number_increment_create.groovy @@ -306,7 +306,7 @@ suite("left_join_range_number_increment_create") { // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -327,9 +327,9 @@ suite("left_join_range_number_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_list_str_increment_create.groovy index 58c2b1c843c189..69067e65719a6a 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_list_str_increment_create.groovy @@ -302,12 +302,12 @@ suite("left_semi_join_list_str_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -328,9 +328,9 @@ suite("left_semi_join_list_str_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy index 6601ef5ea3a40e..789c4aadca7966 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_date_increment_create.groovy @@ -293,12 +293,12 @@ suite("left_semi_join_range_date_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_10] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] def sql_complete_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + def sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -319,9 +319,9 @@ suite("left_semi_join_range_date_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_11, mv_sql_12] + sql_error_list = [mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_10] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_number_increment_create.groovy index 9b606d4f88181a..7f872ae2a9e470 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/left_semi_join_range_number_increment_create.groovy @@ -301,12 +301,13 @@ suite("left_semi_join_range_number_increment_create") { } def sql_all_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] - def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + def sql_increment_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] def sql_complete_list = [] + def sql_error_list = [] // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, primary_tb_change, is_complete_change) @@ -327,9 +328,9 @@ suite("left_semi_join_range_number_increment_create") { // change right table data // create mv base on left table with partition col - sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12] + sql_error_list = [mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12] sql_increment_list = [] - sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8, mv_sql_11] + sql_complete_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_8] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_part_col, slave_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_list_str_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_list_str_increment_create.groovy index 0f8d9514c1ce40..473afc0a58f21d 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_list_str_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_list_str_increment_create.groovy @@ -366,12 +366,15 @@ suite("self_conn_list_str_increment_create") { // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, - mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, + def sql_error_list1 = [mv_sql_1, mv_sql_3, mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12, + mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_18] + def sql_increment_list1 = [mv_sql_4, mv_sql_6] + list_judgement(sql_all_list, sql_increment_list1, sql_complete_list, sql_error_list1, partition_by_part_col, primary_tb_change, is_complete_change) // create mv base on left table with no partition col + def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, + mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_not_part_col, primary_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy index fbc315a9c4ad65..88770b6b56fbcb 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_date_increment_create.groovy @@ -357,11 +357,13 @@ suite("self_conn_range_date_increment_create") { // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, - mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, + def sql_error_list1 = [mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, + mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] + list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list1, partition_by_part_col, primary_tb_change, is_complete_change) + def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, + mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] // create mv base on left table with no partition col list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_not_part_col, primary_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_number_increment_create.groovy b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_number_increment_create.groovy index b74c14bcb8f900..3385db1bf378d0 100644 --- a/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_number_increment_create.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/increment_create/self_conn_range_number_increment_create.groovy @@ -367,11 +367,13 @@ suite("self_conn_range_number_increment_create") { // change left table data // create mv base on left table with partition col - def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, - mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] - list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, + def sql_error_list1 = [mv_sql_1, mv_sql_3, mv_sql_7, mv_sql_9, mv_sql_10, mv_sql_12, + mv_sql_13, mv_sql_15, mv_sql_16, mv_sql_18] + list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list1, partition_by_part_col, primary_tb_change, is_complete_change) + def sql_error_list = [mv_sql_1, mv_sql_3, mv_sql_4, mv_sql_6, mv_sql_7, mv_sql_8, mv_sql_9, mv_sql_10, mv_sql_11, mv_sql_12, + mv_sql_13, mv_sql_14, mv_sql_15, mv_sql_16, mv_sql_17, mv_sql_18] // create mv base on left table with no partition col list_judgement(sql_all_list, sql_increment_list, sql_complete_list, sql_error_list, partition_by_not_part_col, primary_tb_change, is_complete_change) diff --git a/regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.groovy b/regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.groovy new file mode 100644 index 00000000000000..13c10cc03170f1 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/partition_union_rewrite/multi_trace_partition_mv_rewrite.groovy @@ -0,0 +1,426 @@ +// 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("multi_trace_partition_mv_rewrite") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF" + + def initTable = { + sql """ + drop table if exists orders_p + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders_p ( + 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-10-30') INTERVAL 1 DAY + ) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem_p + """ + + // test pre init partition + sql""" + CREATE TABLE IF NOT EXISTS lineitem_p ( + 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-10-30') INTERVAL 1 DAY) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists partsupp + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp ( + ps_partkey INTEGER NOT NULL, + ps_suppkey INTEGER NOT NULL, + ps_availqty INTEGER NOT NULL, + ps_supplycost DECIMALV3(15,2) NOT NULL, + ps_comment VARCHAR(199) NOT NULL + ) + DUPLICATE KEY(ps_partkey, ps_suppkey) + DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql""" + insert into orders_p values + (1, 1, 'ok', 1, '2023-10-17', 'a', 'b', 1, 'yy'), + (1, 1, 'ok', 1, '2023-10-17', 'a', 'b', 1, 'yy'), + (2, 2, 'ok', 1, '2023-10-18', 'c','d',2, 'mm'), + (2, 2, 'ok', 1, '2023-10-18', 'c','d',2, 'mm'), + (2, 2, 'ok', 1, '2023-10-18', 'c','d',2, 'mm'), + (3, 3, 'ok', 1, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-19', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-22', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-22', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-22', 'a', 'b', 1, 'yy'), + (3, 3, 'ok', 1, '2023-10-22', 'a', 'b', 1, 'yy'); + """ + + sql """ + insert into lineitem_p values + (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), + (1, 2, 3, 4, 5.5, 6.5, 2.5, 2.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), + (1, 2, 3, 4, 5.5, 6.5, 3.5, 3.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), + (2, 2, 3, 4, 5.5, 6.5, 4.5, 4.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'), + (2, 2, 3, 4, 5.5, 6.5, 5.5, 5.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'), + (2, 2, 3, 4, 5.5, 6.5, 6.5, 6.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 3, 6, 7.5, 8.5, 7.5, 7.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, 8.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, 9.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-22', '2023-10-22', '2023-10-22', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 11.5, 'k', 'o', '2023-10-22', '2023-10-22', '2023-10-22', 'c', 'd', 'xxxxxxxxx'), + (3, 2, 3, 6, 7.5, 8.5, 9.5, 12.5, 'k', 'o', '2023-10-22', '2023-10-22', '2023-10-22', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into partsupp values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + multi_sql """ + analyze table lineitem_p with sync; + analyze table orders_p with sync; + analyze table partsupp with sync; + """ + + sql """alter table orders_p modify column o_comment set stats ('row_count'='13');""" + sql """alter table lineitem_p modify column l_comment set stats ('row_count'='12');""" + sql """alter table partsupp modify column ps_partkey set stats ('row_count'='2');""" + } + + + def mv_name = "mv_10099" + + def mv_def_sql = """ + select l_shipdate, o_orderdate, ps_partkey, + l_suppkey, sum(o_totalprice) as sum_total + from lineitem_p + inner join orders_p on l_shipdate = o_orderdate + left join partsupp on l_partkey = ps_partkey and l_suppkey = ps_suppkey + group by + l_shipdate, + o_orderdate, + ps_partkey, + l_suppkey; + """ + + def query_all_partition_sql = """ + select l_shipdate, o_orderdate, ps_partkey, + l_suppkey, sum(o_totalprice) as sum_total + from lineitem_p + inner join orders_p on l_shipdate = o_orderdate + left join partsupp on l_partkey = ps_partkey and l_suppkey = ps_suppkey + group by + l_shipdate, + o_orderdate, + ps_partkey, + l_suppkey; + """ + + // partition intersection and union + // lineitem_p add partition, orders_p add partition, partsupp + initTable() + create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") + sql""" + insert into lineitem_p values + (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-20', '2023-10-20', '2023-10-20', 'a', 'b', 'yyyyyyyyy') + """ + sql""" + insert into orders_p values + (1, 1, 'ok', 1.5, '2023-10-20', 'a', 'b', 1, 'yy'), + (1, 1, 'ok', 1.5, '2023-10-21', 'a', 'b', 1, 'yy'); + """ + waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) + waitingPartitionIsExpected(mv_name, "p_20231021_20231022", false) + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_1_0_before "${query_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + + + // lineitem_p add partition, orders_p add partition, partsupp + mv_rewrite_success(query_all_partition_sql, mv_name, + is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) + order_qt_query_1_0_after "${query_all_partition_sql}" + + + // lineitem_p add partition, orders_p delete partition, partsupp + initTable() + create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") + sql""" + insert into lineitem_p values + (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-20', '2023-10-20', '2023-10-20', 'a', 'b', 'yyyyyyyyy') + """ + sql""" + insert into orders_p values + (1, 1, 'ok', 9.5, '2023-10-20', 'a', 'b', 1, 'yy'); + """ + sql """ ALTER TABLE orders_p DROP PARTITION IF EXISTS p_20231017 FORCE; + """ + waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) + waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) + + sql "SET enable_materialized_view_rewrite=false" + order_qt_query_2_0_before "${query_all_partition_sql}" + sql "SET enable_materialized_view_rewrite=true" + + + // lineitem_p add partition, orders_p add partition, partsupp + mv_rewrite_success(query_all_partition_sql, mv_name, + is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) + order_qt_query_2_0_after "${query_all_partition_sql}" + + + // lineitem_p add partition, orders_p modify partition, partsupp + initTable() + create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") + sql""" + insert into lineitem_p values + (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-20', '2023-10-20', '2023-10-20', 'a', 'b', 'yyyyyyyyy') + """ + sql""" + insert into orders_p values + (1, 1, 'ok', 1.5, '2023-10-20', 'a', 'b', 1, 'yy'), + (1, 1, 'ok', 1.5, '2023-10-17', 'a', 'b', 1, 'yy'); + """ + waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) + waitingPartitionIsExpected(mv_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" + + + // lineitem_p add partition, orders_p add partition, partsupp + mv_rewrite_success(query_all_partition_sql, mv_name, + is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) + order_qt_query_3_0_after "${query_all_partition_sql}" +// +// +// // lineitem_p delete partition, orders_p add partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql """ +// ALTER TABLE lineitem_p DROP PARTITION IF EXISTS p_20231017 FORCE; +// """ +// sql""" +// insert into lineitem_p values +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-20', '2023-10-20', '2023-10-20', 'a', 'b', 'yyyyyyyyy') +// """ +// sql""" +// insert into orders_p values +// (1, 1, 'ok', 1.5, '2023-10-20', 'a', 'b', 1, 'yy'); +// """ +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) +// waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) +// +// sql "SET enable_materialized_view_rewrite=false" +// order_qt_query_4_0_before "${query_all_partition_sql}" +// sql "SET enable_materialized_view_rewrite=true" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_4_0_after "${query_all_partition_sql}" +// +// +// // lineitem_p delete partition, orders_p delete partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql """ +// ALTER TABLE lineitem_p DROP PARTITION IF EXISTS p_20231017 FORCE; +// """ +// sql """ +// ALTER TABLE orders_p DROP PARTITION IF EXISTS p_20231018 FORCE; +// """ +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) +// waitingPartitionIsExpected(mv_name, "p_20231018_20231019", false) +// +// sql "SET enable_materialized_view_rewrite=false" +// order_qt_query_5_0_before "${query_all_partition_sql}" +// sql "SET enable_materialized_view_rewrite=true" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_5_0_after "${query_all_partition_sql}" +// +// +// // lineitem_p delete partition, orders_p modify partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql """ +// ALTER TABLE lineitem_p DROP PARTITION IF EXISTS p_20231017 FORCE; +// """ +// sql""" +// insert into lineitem_p values +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-19', '2023-10-19', '2023-10-19', 'a', 'b', 'yyyyyyyyy') +// """ +// sql""" +// insert into orders_p values +// (1, 1, 'ok', 1.5, '2023-10-19', 'a', 'b', 1, 'yy'); +// """ +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) +// waitingPartitionIsExpected(mv_name, "p_20231019_20231020", false) +// +// sql "SET enable_materialized_view_rewrite=false" +// order_qt_query_6_0_before "${query_all_partition_sql}" +// sql "SET enable_materialized_view_rewrite=true" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_6_0_after "${query_all_partition_sql}" +// +// +// +// // lineitem_p modify partition, orders_p add partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql""" +// insert into lineitem_p values +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-20', '2023-10-20', '2023-10-20', 'a', 'b', 'yyyyyyyyy'); +// """ +// sql""" +// insert into orders_p values +// (1, 1, 'ok', 1.5, '2023-10-17', 'a', 'b', 1, 'yy'), +// (1, 1, 'ok', 1.5, '2023-10-20', 'a', 'b', 1, 'yy'); +// """ +// waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", 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" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_7_0_after "${query_all_partition_sql}" +// +// +// // lineitem_p modify partition, orders_p delete partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql""" +// insert into lineitem_p values +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'); +// """ +// sql""" +// insert into orders_p values +// (1, 1, 'ok', 1.5, '2023-10-17', 'a', 'b', 1, 'yy'); +// """ +// sql """ +// ALTER TABLE orders_p DROP PARTITION IF EXISTS p_20231018 FORCE; +// """ +// +// waitingPartitionIsExpected(mv_name, "p_20231020_20231021", false) +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) +// +// sql "SET enable_materialized_view_rewrite=false" +// order_qt_query_8_0_before "${query_all_partition_sql}" +// sql "SET enable_materialized_view_rewrite=true" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_8_0_after "${query_all_partition_sql}" +// +// +// // lineitem_p modify partition, orders_p modify partition, partsupp +// initTable() +// create_async_partition_mv(db, mv_name, mv_def_sql, "(l_shipdate)") +// sql""" +// insert into lineitem_p values +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-17', '2023-10-17', '2023-10-17', 'a', 'b', 'yyyyyyyyy'), +// (1, 2, 3, 4, 5.5, 6.5, 1.5, 1.5, 'o', 'k', '2023-10-18', '2023-10-18', '2023-10-18', 'a', 'b', 'yyyyyyyyy'); +// """ +// sql""" +// insert into orders_p values +// (1, 1, 'ok', 1.5, '2023-10-17', 'a', 'b', 1, 'yy'), +// (1, 1, 'ok', 1.5, '2023-10-18', 'a', 'b', 1, 'yy'); +// """ +// +// waitingPartitionIsExpected(mv_name, "p_20231017_20231018", false) +// waitingPartitionIsExpected(mv_name, "p_20231018_20231019", false) +// +// sql "SET enable_materialized_view_rewrite=false" +// order_qt_query_9_0_before "${query_all_partition_sql}" +// sql "SET enable_materialized_view_rewrite=true" +// +// +// // lineitem_p add partition, orders_p add partition, partsupp +// mv_rewrite_success(query_all_partition_sql, mv_name, +// is_partition_statistics_ready(db, ["lineitem_p", "orders_p", mv_name])) +// order_qt_query_9_0_after "${query_all_partition_sql}" + +} +