diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java new file mode 100644 index 000000000000000..6d8d7c6326c0d9d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/HyperElement.java @@ -0,0 +1,27 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.jobs.joinorder.hypergraph; + +/** + * This is the common base class for all + * */ +public interface HyperElement { + + // Get the references nodes + long getReferenceNodes(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java index 356941694988704..192adf89d413b82 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/edge/Edge.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.edge; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; @@ -32,7 +33,7 @@ /** * Edge in HyperGraph */ -public abstract class Edge { +public abstract class Edge implements HyperElement { private final int index; private final double selectivity; @@ -187,6 +188,7 @@ public boolean isSub(Edge edge) { return LongBitmap.isSubset(getReferenceNodes(), otherBitmap); } + @Override public long getReferenceNodes() { return LongBitmap.newBitmapUnion(leftExtendedNodes, rightExtendedNodes); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java index a4a64e0449deee7..686576de771d94d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/AbstractNode.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.jobs.joinorder.hypergraph.node; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.FilterEdge; @@ -33,7 +34,7 @@ /** * HyperGraph Node. */ -public class AbstractNode { +public class AbstractNode implements HyperElement { protected final int index; protected final List joinEdges; protected final List filterEdges; @@ -65,6 +66,11 @@ public List getEdges() { .build(); } + @Override + public long getReferenceNodes() { + return getNodeMap(); + } + public int getIndex() { return index; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java index 868f97949c07054..3cd0ddd83e3b411 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/HyperGraphComparator.java @@ -19,12 +19,14 @@ import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.joinorder.hypergraph.ConflictRulesMaker; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; import org.apache.doris.nereids.jobs.joinorder.hypergraph.bitmap.LongBitmap; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.FilterEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.JoinEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.ExpressionPosition; import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughJoin; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -41,12 +43,14 @@ import com.google.common.collect.Sets; import java.util.ArrayList; +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.stream.Collectors; import javax.annotation.Nullable; @@ -75,9 +79,9 @@ public class HyperGraphComparator { private final Map> pullUpViewExprWithEdge = new HashMap<>(); private final LogicalCompatibilityContext logicalCompatibilityContext; // this records the slots which needs to reject null - // the key is the target join which should reject null, the value is a pair, the first value of the pair is the - // join type, the second value is also a pair which left represents the slots in the left of join that should - // reject null, right represents the slots in the right of join that should reject null. + // the key is the view join edge which should reject null, the value is a pair, the first value of the pair is the + // query join type, the second value is also a pair which left represents the slots in the left of view join that + // should reject null, right represents the slots in the right of view join that should reject null. private final Map, Set>>> inferredViewEdgeWithCond = new HashMap<>(); private List viewJoinEdgesAfterInferring; private List viewFilterEdgesAfterInferring; @@ -245,9 +249,17 @@ private boolean compareNodeWithExpr(StructInfoNode query, StructInfoNode view) { } int size = queryExprSetList.size(); for (int i = 0; i < size; i++) { - Set mappingQueryExprSet = queryExprSetList.get(i).stream() - .map(logicalCompatibilityContext::getViewNodeExprFromQuery) - .collect(Collectors.toSet()); + Set queryExpressions = queryExprSetList.get(i); + Set mappingQueryExprSet = new HashSet<>(); + for (Expression queryExpression : queryExpressions) { + Optional mappingViewExprByQueryExpr = getMappingViewExprByQueryExpr(queryExpression, query, + this.logicalCompatibilityContext, + ExpressionPosition.NODE); + if (!mappingViewExprByQueryExpr.isPresent()) { + return false; + } + mappingQueryExprSet.add(mappingViewExprByQueryExpr.get()); + } if (!mappingQueryExprSet.equals(viewExprSetList.get(i))) { return false; } @@ -403,7 +415,10 @@ private Map constructQueryToViewJoinMapWithExpr() { if (edgeMap.containsKey(entry.getValue())) { continue; } - Expression viewExpr = logicalCompatibilityContext.getViewJoinExprFromQuery(entry.getKey()); + Expression viewExpr = getMappingViewExprByQueryExpr(entry.getKey(), + entry.getValue(), + logicalCompatibilityContext, + ExpressionPosition.JOIN_EDGE).orElse(null); if (viewExprToEdge.containsKey(viewExpr)) { edgeMap.put(entry.getValue(), Objects.requireNonNull(viewExprToEdge.get(viewExpr))); } @@ -424,7 +439,10 @@ private Map constructQueryToViewFilterMapWithExpr() { if (edgeMap.containsKey(entry.getValue())) { continue; } - Expression viewExpr = logicalCompatibilityContext.getViewFilterExprFromQuery(entry.getKey()); + Expression viewExpr = getMappingViewExprByQueryExpr(entry.getKey(), + entry.getValue(), + logicalCompatibilityContext, + ExpressionPosition.FILTER_EDGE).orElse(null); if (viewExprToEdge.containsKey(viewExpr)) { edgeMap.put(entry.getValue(), Objects.requireNonNull(viewExprToEdge.get(viewExpr))); } @@ -463,17 +481,17 @@ private boolean compareEdgeWithNode(Edge query, Edge view) { } private boolean compareFilterEdgeWithNode(FilterEdge query, FilterEdge view) { - return rewriteQueryNodeMap(query.getReferenceNodes()) == view.getReferenceNodes(); + return getViewNodesByQuery(query.getReferenceNodes()) == view.getReferenceNodes(); } private boolean compareJoinEdgeWithNode(JoinEdge query, JoinEdge view) { boolean res = false; if (query.getJoinType().swap() == view.getJoinType()) { - res |= rewriteQueryNodeMap(query.getLeftExtendedNodes()) == view.getRightExtendedNodes() - && rewriteQueryNodeMap(query.getRightExtendedNodes()) == view.getLeftExtendedNodes(); + res |= getViewNodesByQuery(query.getLeftExtendedNodes()) == view.getRightExtendedNodes() + && getViewNodesByQuery(query.getRightExtendedNodes()) == view.getLeftExtendedNodes(); } - res |= rewriteQueryNodeMap(query.getLeftExtendedNodes()) == view.getLeftExtendedNodes() - && rewriteQueryNodeMap(query.getRightExtendedNodes()) == view.getRightExtendedNodes(); + res |= getViewNodesByQuery(query.getLeftExtendedNodes()) == view.getLeftExtendedNodes() + && getViewNodesByQuery(query.getRightExtendedNodes()) == view.getRightExtendedNodes(); return res; } @@ -496,8 +514,8 @@ private boolean compareJoinEdgeOrInfer(JoinEdge query, JoinEdge view) { } private boolean tryInferEdge(JoinEdge query, JoinEdge view) { - if (rewriteQueryNodeMap(query.getLeftRequiredNodes()) != view.getLeftRequiredNodes() - || rewriteQueryNodeMap(query.getRightRequiredNodes()) != view.getRightRequiredNodes()) { + if (getViewNodesByQuery(query.getLeftRequiredNodes()) != view.getLeftRequiredNodes() + || getViewNodesByQuery(query.getRightRequiredNodes()) != view.getRightRequiredNodes()) { return false; } if (!query.getJoinType().equals(view.getJoinType())) { @@ -518,7 +536,7 @@ private boolean tryInferEdge(JoinEdge query, JoinEdge view) { return true; } - private long rewriteQueryNodeMap(long bitmap) { + private long getViewNodesByQuery(long bitmap) { long newBitmap = LongBitmap.newBitmap(); for (int i : LongBitmap.getIterator(bitmap)) { int newIdx = getQueryToViewNodeIdMap().getOrDefault(i, 0); @@ -527,6 +545,35 @@ private long rewriteQueryNodeMap(long bitmap) { return newBitmap; } + private Optional getMappingViewExprByQueryExpr(Expression queryExpression, + HyperElement queryExpressionBelongedHyperElement, + LogicalCompatibilityContext context, + ExpressionPosition expressionPosition) { + Expression queryShuttledExpr; + Collection> viewExpressions; + if (ExpressionPosition.JOIN_EDGE.equals(expressionPosition)) { + queryShuttledExpr = context.getQueryJoinShuttledExpr(queryExpression); + viewExpressions = context.getViewJoinExprFromQuery(queryShuttledExpr); + } else if (ExpressionPosition.FILTER_EDGE.equals(expressionPosition)) { + queryShuttledExpr = context.getQueryFilterShuttledExpr(queryExpression); + viewExpressions = context.getViewFilterExprFromQuery(queryShuttledExpr); + } else { + queryShuttledExpr = context.getQueryNodeShuttledExpr(queryExpression); + viewExpressions = context.getViewNodeExprFromQuery(queryShuttledExpr); + } + if (viewExpressions.size() == 1) { + return Optional.of(viewExpressions.iterator().next().key()); + } + long queryReferenceNodes = queryExpressionBelongedHyperElement.getReferenceNodes(); + long viewReferenceNodes = getViewNodesByQuery(queryReferenceNodes); + for (Pair viewExpressionPair : viewExpressions) { + if (viewExpressionPair.value().getReferenceNodes() == viewReferenceNodes) { + return Optional.of(viewExpressionPair.key()); + } + } + return Optional.empty(); + } + private void compareJoinEdgeWithExpr(Edge query, Edge view) { Set queryExprSet = query.getExpressionSet(); Set viewExprSet = view.getExpressionSet(); @@ -534,7 +581,10 @@ private void compareJoinEdgeWithExpr(Edge query, Edge view) { Set exprMappedOfView = new HashSet<>(); List residualQueryExpr = new ArrayList<>(); for (Expression queryExpr : queryExprSet) { - Expression viewExpr = logicalCompatibilityContext.getViewJoinExprFromQuery(queryExpr); + Expression viewExpr = getMappingViewExprByQueryExpr(queryExpr, + query, + logicalCompatibilityContext, + ExpressionPosition.JOIN_EDGE).orElse(null); if (viewExprSet.contains(viewExpr)) { exprMappedOfView.add(viewExpr); } else { @@ -553,7 +603,10 @@ private void compareFilterEdgeWithExpr(Edge query, Edge view) { Set exprMappedOfView = new HashSet<>(); List residualQueryExpr = new ArrayList<>(); for (Expression queryExpr : queryExprSet) { - Expression viewExpr = logicalCompatibilityContext.getViewFilterExprFromQuery(queryExpr); + Expression viewExpr = getMappingViewExprByQueryExpr(queryExpr, + query, + logicalCompatibilityContext, + ExpressionPosition.FILTER_EDGE).orElse(null); if (viewExprSet.contains(viewExpr)) { exprMappedOfView.add(viewExpr); } else { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java index ca13c9701dabc2c..77ab37873d06b40 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/LogicalCompatibilityContext.java @@ -17,6 +17,8 @@ package org.apache.doris.nereids.rules.exploration.mv; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.rules.exploration.mv.StructInfo.ExpressionPosition; @@ -36,8 +38,10 @@ import com.google.common.base.Suppliers; import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; -import java.util.HashMap; +import java.util.Collection; import java.util.Map; import java.util.function.Supplier; @@ -48,11 +52,15 @@ public class LogicalCompatibilityContext { private final BiMap queryToViewNodeMapping; private final BiMap queryToViewNodeIDMapping; private final ObjectId planNodeId; - private final Supplier> queryToViewJoinEdgeExpressionMappingSupplier; - private final Supplier> queryToViewNodeExpressionMappingSupplier; - private final Supplier> queryToViewFilterEdgeExpressionMappingSupplier; - @Deprecated - private BiMap queryToViewAllExpressionMapping; + private final Supplier>> + queryToViewJoinEdgeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledJoinExpressionMappingSupplier; + private final Supplier>> + queryToViewNodeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledNodeExpressionMappingSupplier; + private final Supplier>> + queryToViewFilterEdgeExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledFilterExpressionMappingSupplier; /** * LogicalCompatibilityContext @@ -66,16 +74,25 @@ private LogicalCompatibilityContext(BiMap queryT queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.JOIN_EDGE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.JOIN_EDGE))); + this.queryToQueryShuttledJoinExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.JOIN_EDGE)); + this.queryToViewNodeExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE))); + this.queryToQueryShuttledNodeExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.NODE)); + this.queryToViewFilterEdgeExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.FILTER_EDGE), viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.FILTER_EDGE))); + this.queryToQueryShuttledFilterExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.FILTER_EDGE)); + this.queryToViewNodeMapping = queryToViewNodeMapping; this.queryToViewNodeIDMapping = HashBiMap.create(); queryToViewNodeMapping.forEach((k, v) -> queryToViewNodeIDMapping.put(k.getIndex(), v.getIndex())); @@ -92,18 +109,30 @@ public BiMap getQueryToViewNodeIDMapping() { return queryToViewNodeIDMapping; } - public Expression getViewJoinExprFromQuery(Expression queryJoinExpr) { + public Collection> getViewJoinExprFromQuery(Expression queryJoinExpr) { return queryToViewJoinEdgeExpressionMappingSupplier.get().get(queryJoinExpr); } - public Expression getViewFilterExprFromQuery(Expression queryJoinExpr) { + public Expression getQueryJoinShuttledExpr(Expression queryJoinExpr) { + return queryToQueryShuttledJoinExpressionMappingSupplier.get().get(queryJoinExpr); + } + + public Collection> getViewFilterExprFromQuery(Expression queryJoinExpr) { return queryToViewFilterEdgeExpressionMappingSupplier.get().get(queryJoinExpr); } - public Expression getViewNodeExprFromQuery(Expression queryJoinExpr) { + public Expression getQueryFilterShuttledExpr(Expression queryFilterExpr) { + return queryToQueryShuttledFilterExpressionMappingSupplier.get().get(queryFilterExpr); + } + + public Collection> getViewNodeExprFromQuery(Expression queryJoinExpr) { return queryToViewNodeExpressionMappingSupplier.get().get(queryJoinExpr); } + public Expression getQueryNodeShuttledExpr(Expression queryNodeExpr) { + return queryToQueryShuttledNodeExpressionMappingSupplier.get().get(queryNodeExpr); + } + /** * Generate logical compatibility context, * this make expression mapping between query and view by relation and the slot in relation mapping @@ -134,24 +163,31 @@ public static LogicalCompatibilityContext from(RelationMapping relationMapping, viewStructInfo); } - private static BiMap generateExpressionMapping( + /** + * The result is multimap + * the key is shuttled query expr + * the value is original view expr collection + * */ + private static Multimap> generateExpressionMapping( Map viewToQuerySlotMapping, - Map queryShuttledExprToExprMap, - Map viewShuttledExprToExprMap) { - final Map viewEdgeToConjunctsMapQueryBased = new HashMap<>(); - BiMap queryToViewEdgeMapping = HashBiMap.create(); + Multimap> queryShuttledExprToExprMap, + Multimap> viewShuttledExprToExprMap) { + Multimap> queryToViewEdgeMapping = HashMultimap.create(); if (queryShuttledExprToExprMap == null || viewShuttledExprToExprMap == null || queryShuttledExprToExprMap.isEmpty() || viewShuttledExprToExprMap.isEmpty()) { return queryToViewEdgeMapping; } + final Multimap> viewShuttledExprToExprMapQueryBased = + HashMultimap.create(); viewShuttledExprToExprMap.forEach((shuttledExpr, expr) -> { - viewEdgeToConjunctsMapQueryBased.put( + viewShuttledExprToExprMapQueryBased.put( orderSlotAsc(ExpressionUtils.replace(shuttledExpr, viewToQuerySlotMapping)), expr); }); - queryShuttledExprToExprMap.forEach((exprSet, edge) -> { - Expression viewExpr = viewEdgeToConjunctsMapQueryBased.get(orderSlotAsc(exprSet)); - if (viewExpr != null) { - queryToViewEdgeMapping.put(edge, viewExpr); + queryShuttledExprToExprMap.forEach((shuttledExpr, expr) -> { + Collection> viewExpressions = viewShuttledExprToExprMapQueryBased.get( + orderSlotAsc(shuttledExpr)); + if (viewExpressions != null) { + queryToViewEdgeMapping.putAll(shuttledExpr, viewExpressions); } }); return queryToViewEdgeMapping; 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 526ec7030d2db59..1edb3f95a7315ab 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 @@ -23,10 +23,10 @@ import org.apache.doris.mtmv.BaseTableInfo; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.executor.Rewriter; +import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperElement; import org.apache.doris.nereids.jobs.joinorder.hypergraph.HyperGraph; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.JoinEdge; import org.apache.doris.nereids.jobs.joinorder.hypergraph.node.StructInfoNode; -import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; import org.apache.doris.nereids.trees.copier.DeepCopierContext; @@ -61,12 +61,15 @@ import org.apache.doris.nereids.trees.plans.visitor.ExpressionLineageReplacer; import org.apache.doris.nereids.util.ExpressionUtils; +import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Multimap; import com.google.common.collect.Sets; import java.util.ArrayList; import java.util.BitSet; +import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; @@ -109,9 +112,23 @@ public class StructInfo { // split predicates is shuttled private SplitPredicate splitPredicate; private EquivalenceClass equivalenceClass; - // Key is the expression shuttled and the value is the origin expression + // For value of Map, the key is the position of expression + // the value is the expressions and the hyper element of expression pair + // Key of pair is the expression shuttled and the value is the origin expression and the hyper element it belonged + // Sometimes origin expressions are different and shuttled expression is same + // Such as origin expressions are l_partkey#0 > 1 and l_partkey#10 > 1 and shuttled expression is l_partkey#10 > 1 // this is for building LogicalCompatibilityContext later. - private final Map> shuttledExpressionsToExpressionsMap; + private final Map>> + shuttledExpressionsToExpressionsMap; + // For value of Map, the key is the position of expression + // the value is the original expression and shuttled expression map + // Such as origin expressions are l_partkey#0 > 1 and shuttled expression is l_partkey#10 > 1 + // the map would be {ExpressionPosition.FILTER, { + // l_partkey#0 > 1 : l_partkey#10 > 1 + // }} + // this is for building LogicalCompatibilityContext later. + private final Map> expressionToShuttledExpressionToMap; + // Record the exprId and the corresponding expr map, this is used by expression shuttled private final Map namedExprIdAndExprMapping; private final List planOutputShuttledExpressions; @@ -123,7 +140,9 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG Plan bottomPlan, List relations, Map relationIdStructInfoNodeMap, @Nullable Predicates predicates, - Map> shuttledExpressionsToExpressionsMap, + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionToShuttledExpressionToMap, Map namedExprIdAndExprMapping, BitSet tableIdSet, SplitPredicate splitPredicate, @@ -142,6 +161,7 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG this.splitPredicate = splitPredicate; this.equivalenceClass = equivalenceClass; this.shuttledExpressionsToExpressionsMap = shuttledExpressionsToExpressionsMap; + this.expressionToShuttledExpressionToMap = expressionToShuttledExpressionToMap; this.namedExprIdAndExprMapping = namedExprIdAndExprMapping; this.planOutputShuttledExpressions = planOutputShuttledExpressions; } @@ -152,7 +172,8 @@ private StructInfo(Plan originalPlan, ObjectId originalPlanId, HyperGraph hyperG public StructInfo withPredicates(Predicates predicates) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, predicates, - this.shuttledExpressionsToExpressionsMap, this.namedExprIdAndExprMapping, this.tableBitSet, + this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, + this.namedExprIdAndExprMapping, this.tableBitSet, null, null, this.planOutputShuttledExpressions); } @@ -162,13 +183,16 @@ public StructInfo withPredicates(Predicates predicates) { public StructInfo withTableBitSet(BitSet tableBitSet) { return new StructInfo(this.originalPlan, this.originalPlanId, this.hyperGraph, this.valid, this.topPlan, this.bottomPlan, this.relations, this.relationIdStructInfoNodeMap, this.predicates, - this.shuttledExpressionsToExpressionsMap, this.namedExprIdAndExprMapping, tableBitSet, + this.shuttledExpressionsToExpressionsMap, this.expressionToShuttledExpressionToMap, + this.namedExprIdAndExprMapping, tableBitSet, this.splitPredicate, this.equivalenceClass, this.planOutputShuttledExpressions); } private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, Plan topPlan, - Map> shuttledExpressionsToExpressionsMap, + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionToShuttledExpressionToMap, Map namedExprIdAndExprMapping, List relations, Map relationIdStructInfoNodeMap, @@ -196,8 +220,9 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, structInfoNode.getPlan().accept(ExpressionLineageReplacer.INSTANCE, replaceContext); // Replace expressions by expression map List replacedExpressions = replaceContext.getReplacedExpressions(); - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, - ExpressionPosition.NODE, replacedExpressions.get(0), expression); + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, + ExpressionPosition.NODE, replacedExpressions.get(0), expression, node); // Record this, will be used in top level expression shuttle later, see the method // ExpressionLineageReplacer#visitGroupPlan namedExprIdAndExprMapping.putAll(replaceContext.getExprIdExpressionMap()); @@ -223,8 +248,10 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, // Replace expressions by expression map List replacedExpressions = replaceContext.getReplacedExpressions(); for (int i = 0; i < replacedExpressions.size(); i++) { - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, - ExpressionPosition.JOIN_EDGE, replacedExpressions.get(i), joinConjunctExpressions.get(i)); + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, + ExpressionPosition.JOIN_EDGE, replacedExpressions.get(i), joinConjunctExpressions.get(i), + edge); } // Record this, will be used in top level expression shuttle later, see the method // ExpressionLineageReplacer#visitGroupPlan @@ -236,10 +263,11 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, filterExpressions.forEach(predicate -> { // this is used for LogicalCompatibilityContext ExpressionUtils.extractConjunction(predicate).forEach(expr -> - putShuttledExpressionsToExpressionsMap(shuttledExpressionsToExpressionsMap, + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, ExpressionPosition.FILTER_EDGE, ExpressionUtils.shuttleExpressionWithLineage(predicate, topPlan, new BitSet()), - predicate)); + predicate, filterEdge)); }); }); return true; @@ -311,11 +339,13 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable // collect struct info fromGraph List relationList = new ArrayList<>(); Map relationIdStructInfoNodeMap = new LinkedHashMap<>(); - Map> shuttledHashConjunctsToConjunctsMap = - new LinkedHashMap<>(); + Map>> + shuttledHashConjunctsToConjunctsMap = new LinkedHashMap<>(); Map namedExprIdAndExprMapping = new LinkedHashMap<>(); BitSet tableBitSet = new BitSet(); + Map> expressionToShuttledExpressionToMap = new HashMap<>(); boolean valid = collectStructInfoFromGraph(hyperGraph, topPlan, shuttledHashConjunctsToConjunctsMap, + expressionToShuttledExpressionToMap, namedExprIdAndExprMapping, relationList, relationIdStructInfoNodeMap, @@ -332,19 +362,11 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable ExpressionUtils.shuttleExpressionWithLineage(originalPlan.getOutput(), originalPlan, new BitSet()); return new StructInfo(originalPlan, originalPlanId, hyperGraph, valid, topPlan, bottomPlan, relationList, relationIdStructInfoNodeMap, predicates, shuttledHashConjunctsToConjunctsMap, + expressionToShuttledExpressionToMap, namedExprIdAndExprMapping, tableBitSet, null, null, planOutputShuttledExpressions); } - /** - * Build Struct info from group. - * Maybe return multi structInfo when original plan already be rewritten by mv - */ - public static StructInfo of(Group group) { - // TODO build graph from original plan and get relations and predicates from graph - return null; - } - public List getRelations() { return relations; } @@ -401,21 +423,36 @@ public Map getRelationIdStructInfoNodeMap() { return relationIdStructInfoNodeMap; } - public Map> getShuttledExpressionsToExpressionsMap() { + public Map>> + getShuttledExpressionsToExpressionsMap() { return shuttledExpressionsToExpressionsMap; } - private static void putShuttledExpressionsToExpressionsMap( - Map> shuttledExpressionsToExpressionsMap, + public Map> getExpressionToShuttledExpressionToMap() { + return expressionToShuttledExpressionToMap; + } + + private static void putShuttledExpressionToExpressionsMap( + Map>> + shuttledExpressionsToExpressionsMap, + Map> expressionPositionToExpressionToMap, ExpressionPosition expressionPosition, - Expression key, Expression value) { - Map expressionExpressionMap = shuttledExpressionsToExpressionsMap.get( - expressionPosition); - if (expressionExpressionMap == null) { - expressionExpressionMap = new LinkedHashMap<>(); - shuttledExpressionsToExpressionsMap.put(expressionPosition, expressionExpressionMap); - } - expressionExpressionMap.put(key, value); + Expression shuttledExpression, Expression originalExpression, HyperElement valueBelongedElement) { + Multimap> shuttledExpressionToExpressionMap = + shuttledExpressionsToExpressionsMap.get(expressionPosition); + if (shuttledExpressionToExpressionMap == null) { + shuttledExpressionToExpressionMap = HashMultimap.create(); + shuttledExpressionsToExpressionsMap.put(expressionPosition, shuttledExpressionToExpressionMap); + } + shuttledExpressionToExpressionMap.put(shuttledExpression, Pair.of(originalExpression, valueBelongedElement)); + + Map originalExprToShuttledExprMap = + expressionPositionToExpressionToMap.get(expressionPosition); + if (originalExprToShuttledExprMap == null) { + originalExprToShuttledExprMap = new HashMap<>(); + expressionPositionToExpressionToMap.put(expressionPosition, originalExprToShuttledExprMap); + } + originalExprToShuttledExprMap.put(originalExpression, shuttledExpression); } public List getExpressions() { diff --git a/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out index 3b9c3a1219ea15c..e192959a03d4c24 100644 --- a/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out +++ b/regression-test/data/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.out @@ -315,3 +315,27 @@ a 3 3 a,a,a 4.0 yy 3 1 a 4 2 a,a 4.0 yy 2 1 c 3 6 c,c,c 5.333333333333333 mi 3 2 +-- !query29_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query29_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query30_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query30_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query31_0_before -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + +-- !query31_0_after -- +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 \N \N 8 8 1 +1 2023-12-09 1 yy 2 2 2 4 3 \N 2 3 1 2 8 8 1 + diff --git a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy index 9d60280503cf748..183ad259686ad88 100644 --- a/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/agg_without_roll_up/aggregate_without_roll_up.groovy @@ -36,7 +36,8 @@ suite("aggregate_without_roll_up") { o_orderpriority CHAR(15) NOT NULL, o_clerk CHAR(15) NOT NULL, o_shippriority INTEGER NOT NULL, - o_comment VARCHAR(79) NOT NULL + o_comment VARCHAR(79) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(o_orderkey, o_custkey) DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 @@ -44,11 +45,9 @@ suite("aggregate_without_roll_up") { "replication_num" = "1" ); """ - sql """ drop table if exists lineitem """ - sql""" CREATE TABLE IF NOT EXISTS lineitem ( l_orderkey INTEGER NOT NULL, @@ -66,7 +65,8 @@ suite("aggregate_without_roll_up") { l_receiptdate DATE NOT NULL, l_shipinstruct CHAR(25) NOT NULL, l_shipmode CHAR(10) NOT NULL, - l_comment VARCHAR(44) NOT NULL + l_comment VARCHAR(44) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 @@ -74,18 +74,17 @@ suite("aggregate_without_roll_up") { "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 + ps_comment VARCHAR(199) NOT NULL, + public_col INT NULL ) DUPLICATE KEY(ps_partkey, ps_suppkey) DISTRIBUTED BY HASH(ps_partkey) BUCKETS 3 @@ -96,29 +95,29 @@ suite("aggregate_without_roll_up") { sql """ insert into lineitem values - (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), - (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', 1), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', null), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', 2), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy', null), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx', 3); """ sql """ insert into orders values - (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy'), - (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy'), - (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy'), - (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy'), - (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), - (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), - (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + (1, 1, 'o', 9.5, '2023-12-08', 'a', 'b', 1, 'yy', 1), + (1, 1, 'o', 10.5, '2023-12-08', 'a', 'b', 1, 'yy', null), + (2, 1, 'o', 11.5, '2023-12-09', 'a', 'b', 1, 'yy', 2), + (3, 1, 'o', 12.5, '2023-12-10', 'a', 'b', 1, 'yy', null), + (3, 1, 'o', 33.5, '2023-12-10', 'a', 'b', 1, 'yy', 3), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm', null), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi', 4), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi', null); """ sql """ insert into partsupp values - (2, 3, 9, 10.01, 'supply1'), - (2, 3, 10, 11.01, 'supply2'); + (2, 3, 9, 10.01, 'supply1', 1), + (2, 3, 10, 11.01, 'supply2', null); """ // single table @@ -1356,4 +1355,463 @@ suite("aggregate_without_roll_up") { """ async_mv_rewrite_fail(db, mv27_0, query27_0, "mv27_0") sql """ DROP MATERIALIZED VIEW IF EXISTS mv27_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + def mv29_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query29_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query29_0_before "${query29_0}" + async_mv_rewrite_success(db, mv29_0, query29_0, "mv29_0") + order_qt_query29_0_after "${query29_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + // mv join condition has alias + def mv30_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey_alias, + orders.public_col as col1, + l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey_alias, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2, + sum( + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey as o_orderkey_alias, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey as l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on lineitem.l_orderkey_alias = orders.o_orderkey_alias + inner join ( + select + ps_partkey as ps_partkey_alias, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on partsupp.ps_partkey_alias = orders.o_orderkey_alias + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey_alias = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query30_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query30_0_before "${query30_0}" + async_mv_rewrite_success(db, mv30_0, query30_0, "mv30_0") + order_qt_query30_0_after "${query30_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv30_0""" + + + // query and mv has the same filter but position is different, should rewrite successfully + // query join condition has alias + def mv31_0 = """ + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as col1, + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey + l_orderkey + ps_partkey * 2, + sum( + o_orderkey + l_orderkey + ps_partkey * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on l_orderkey = o_orderkey + inner join ( + select + ps_partkey, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on ps_partkey = o_orderkey + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + def query31_0 = """ +select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey_alias, + orders.public_col as col1, + l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as col2, + ps_partkey_alias, + ps_suppkey, + partsupp.public_col as col3, + partsupp.public_col * 2 as col4, + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2, + sum( + o_orderkey_alias + l_orderkey_alias + ps_partkey_alias * 2 + ), + count() as count_all + from + ( + select + o_custkey, + o_orderdate, + o_shippriority, + o_comment, + o_orderkey as o_orderkey_alias, + orders.public_col as public_col + from + orders + ) orders + left join ( + select + l_orderkey as l_orderkey_alias, + l_partkey, + l_suppkey, + lineitem.public_col as public_col + from + lineitem + where + lineitem.public_col is null + or lineitem.public_col <> 1 + ) lineitem on lineitem.l_orderkey_alias = orders.o_orderkey_alias + inner join ( + select + ps_partkey as ps_partkey_alias, + ps_suppkey, + partsupp.public_col as public_col + from + partsupp + ) partsupp on partsupp.ps_partkey_alias = orders.o_orderkey_alias + where + lineitem.public_col is null + or lineitem.public_col <> 1 + and o_orderkey_alias = 2 + group by + 1, + 2, + 3, + 4, + 5, + 6, + 7, + 8, + 9, + 10, + 11, + 12, + 13, + 14; + """ + order_qt_query31_0_before "${query31_0}" + async_mv_rewrite_success(db, mv31_0, query31_0, "mv31_0") + order_qt_query31_0_after "${query31_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv31_0""" }