diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/StructInfoNode.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/StructInfoNode.java index 28f1ed405e82ef..3866bef788eade 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/StructInfoNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/joinorder/hypergraph/node/StructInfoNode.java @@ -19,6 +19,9 @@ import org.apache.doris.common.Pair; import org.apache.doris.nereids.jobs.joinorder.hypergraph.edge.Edge; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PredicateCollectorContext; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.Plan; @@ -27,6 +30,7 @@ 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.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -39,26 +43,42 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import javax.annotation.Nullable; /** * HyperGraph Node. */ public class StructInfoNode extends AbstractNode { - private final List> expressions; + private final Pair>, List>> expressions; private final Set relationSet; + /** + * the constructor of StructInfoNode + */ public StructInfoNode(int index, Plan plan, List edges) { super(extractPlan(plan), index, edges); relationSet = plan.collect(CatalogRelation.class::isInstance); - expressions = collectExpressions(plan); + // check the node pattern is valid + PlanCheckContext checkContext = PlanCheckContext.of(ImmutableSet.of()); + Boolean checkResult = plan.accept(StructInfo.PLAN_PATTERN_CHECKER, checkContext); + if (checkResult && !checkContext.isWindowUnderAggregate() && checkContext.getTopAggregateNum() <= 1 + && checkContext.getTopWindowNum() <= 1) { + expressions = collectExpressions(plan); + } else { + expressions = null; + } } public StructInfoNode(int index, Plan plan) { this(index, plan, new ArrayList<>()); } - private @Nullable List> collectExpressions(Plan plan) { + /** + * The key of pair is the expression which could be moved around anywhere, + * the value of pair is the expression which could not be moved around anywhere. + * */ + private @Nullable Pair>, List>> collectExpressions(Plan plan) { Pair>> collector = Pair.of(true, ImmutableList.builder()); plan.accept(new DefaultPlanVisitor>>>() { @@ -69,7 +89,6 @@ public Void visitLogicalAggregate(LogicalAggregate aggregate, return null; } collector.value().add(ImmutableSet.copyOf(aggregate.getExpressions())); - collector.value().add(ImmutableSet.copyOf(((LogicalAggregate) plan).getGroupByExpressions())); return super.visit(aggregate, collector); } @@ -93,6 +112,13 @@ public Void visitGroupPlan(GroupPlan groupPlan, return groupActualPlan.accept(this, collector); } + @Override + public Void visitLogicalWindow(LogicalWindow window, + Pair>> context) { + collector.value().add(ImmutableSet.copyOf(window.getActualWindowExpressions())); + return super.visit(window, context); + } + @Override public Void visit(Plan plan, Pair>> context) { if (!isValidNodePlan(plan)) { @@ -102,24 +128,52 @@ public Void visit(Plan plan, Pair return super.visit(plan, context); } }, collector); - return collector.key() ? collector.value().build() : null; + + if (!collector.key()) { + return null; + } + PredicateCollectorContext predicateCollectorContext = new PredicateCollectorContext(); + plan.accept(StructInfo.PREDICATE_COLLECTOR, predicateCollectorContext); + collector.value().add(ImmutableSet.copyOf(predicateCollectorContext.getCouldPullUpPredicates())); + if (predicateCollectorContext.getCouldNotPullUpPredicates().isEmpty()) { + return Pair.of(collector.value().build(), ImmutableList.of()); + } + return Pair.of(collector.value().build(), + ImmutableList.of(predicateCollectorContext.getCouldNotPullUpPredicates())); } private boolean isValidNodePlan(Plan plan) { return plan instanceof LogicalProject || plan instanceof LogicalAggregate - || plan instanceof LogicalFilter || plan instanceof LogicalCatalogRelation; + || plan instanceof LogicalFilter || plan instanceof LogicalCatalogRelation + || plan instanceof LogicalWindow; } /** * get all expressions of nodes */ public @Nullable List getExpressions() { - return expressions == null ? null : expressions.stream() - .flatMap(Collection::stream) - .collect(Collectors.toList()); + if (expressions == null) { + return null; + } + return Stream.concat(expressions.key().stream().flatMap(Collection::stream), + expressions.value().stream().flatMap(Collection::stream)).collect(Collectors.toList()); + } + + public @Nullable List getCouldMoveExpressions() { + if (expressions == null) { + return null; + } + return expressions.key().stream().flatMap(Collection::stream).collect(Collectors.toList()); + } + + public @Nullable List getCouldNotMoveExpressions() { + if (expressions == null) { + return null; + } + return expressions.value().stream().flatMap(Collection::stream).collect(Collectors.toList()); } - public @Nullable List> getExprSetList() { + public @Nullable Pair>, List>> getExprSetList() { return expressions; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index 6e47834d6c92ef..f361348ccad9d6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -47,6 +47,9 @@ import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterScanRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectScanRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewWindowAggregateRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewWindowJoinRule; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewWindowScanRule; import org.apache.doris.nereids.rules.expression.ExpressionNormalizationAndOptimization; import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; @@ -268,6 +271,9 @@ public class RuleSet { .add(MaterializedViewProjectFilterScanRule.INSTANCE) .add(MaterializedViewAggregateOnNoneAggregateRule.INSTANCE) .add(MaterializedViewOnlyScanRule.INSTANCE) + .add(MaterializedViewWindowScanRule.INSTANCE) + .add(MaterializedViewWindowJoinRule.INSTANCE) + .add(MaterializedViewWindowAggregateRule.INSTANCE) .build(); public static final List MATERIALIZED_VIEW_IN_RBO_RULES = planRuleFactories() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 37e357d8f0d6fd..b721789a38d254 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -451,6 +451,27 @@ public enum RuleType { MATERIALIZED_VIEW_FILTER_PROJECT_JOIN(RuleTypeClass.MATERIALIZE_VIEW), MATERIALIZED_VIEW_ONLY_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + // window above join without aggregate + MATERIALIZED_VIEW_PROJECT_WINDOW_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_FILTER_WINDOW_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_ONLY_WINDOW_JOIN(RuleTypeClass.MATERIALIZE_VIEW), + + // window above scan without aggregate + MATERIALIZED_VIEW_FILTER_WINDOW_SCAN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_PROJECT_WINDOW_SCAN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_SCAN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_SCAN(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_ONLY_WINDOW_SCAN(RuleTypeClass.MATERIALIZE_VIEW), + + // window with aggregate + MATERIALIZED_VIEW_PROJECT_WINDOW_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_FILTER_WINDOW_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_ONLY_WINDOW_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), + MATERIALIZED_VIEW_PROJECT_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), MATERIALIZED_VIEW_FILTER_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), MATERIALIZED_VIEW_PROJECT_FILTER_AGGREGATE(RuleTypeClass.MATERIALIZE_VIEW), 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..e6e7f395d9b20d 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 @@ -683,7 +683,8 @@ protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext casca PlanCheckContext checkContext = PlanCheckContext.of(SUPPORTED_JOIN_TYPE_SET); // if query or mv contains more then one top aggregate, should fail return structInfo.getTopPlan().accept(StructInfo.PLAN_PATTERN_CHECKER, checkContext) - && checkContext.isContainsTopAggregate() && checkContext.getTopAggregateNum() <= 1; + && checkContext.isContainsTopAggregate() && checkContext.getTopAggregateNum() <= 1 + && !checkContext.isContainsTopWindow(); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java index 122688b766e3a6..cf27aa2982a662 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java @@ -81,6 +81,6 @@ protected Plan rewriteQueryByView(MatchMode matchMode, protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext cascadesContext) { PlanCheckContext checkContext = PlanCheckContext.of(SUPPORTED_JOIN_TYPE_SET); return structInfo.getTopPlan().accept(StructInfo.PLAN_PATTERN_CHECKER, checkContext) - && !checkContext.isContainsTopAggregate(); + && !checkContext.isContainsTopAggregate() && !checkContext.isContainsTopWindow(); } } 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..4c9b71e81fad66 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 @@ -282,7 +282,8 @@ protected List doRewrite(StructInfo queryStructInfo, CascadesContext casca rewrittenPlan = new LogicalFilter<>(Sets.newLinkedHashSet(rewriteCompensatePredicates), mvScan); } boolean checkResult = rewriteQueryByViewPreCheck(matchMode, queryStructInfo, - viewStructInfo, viewToQuerySlotMapping, rewrittenPlan, materializationContext); + viewStructInfo, viewToQuerySlotMapping, rewrittenPlan, materializationContext, + comparisonResult); if (!checkResult) { continue; } @@ -517,7 +518,7 @@ protected Pair>, Map>> */ protected boolean rewriteQueryByViewPreCheck(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, - MaterializationContext materializationContext) { + MaterializationContext materializationContext, ComparisonResult comparisonResult) { if (materializationContext instanceof SyncMaterializationContext && queryStructInfo.getBottomPlan() instanceof LogicalOlapScan) { LogicalOlapScan olapScan = (LogicalOlapScan) queryStructInfo.getBottomPlan(); @@ -741,13 +742,13 @@ protected SplitPredicate predicatesCompensate( // set pulled up expression to queryStructInfo predicates and update related predicates if (!queryPulledUpExpressions.isEmpty()) { queryStructInfo = queryStructInfo.withPredicates( - queryStructInfo.getPredicates().merge(queryPulledUpExpressions)); + queryStructInfo.getPredicates().mergePulledUpPredicates(queryPulledUpExpressions)); } List viewPulledUpExpressions = ImmutableList.copyOf(comparisonResult.getViewExpressions()); // set pulled up expression to viewStructInfo predicates and update related predicates if (!viewPulledUpExpressions.isEmpty()) { viewStructInfo = viewStructInfo.withPredicates( - viewStructInfo.getPredicates().merge(viewPulledUpExpressions)); + viewStructInfo.getPredicates().mergePulledUpPredicates(viewPulledUpExpressions)); } // if the join type in query and mv plan is different, we should check query is have the // filters which rejects null @@ -760,8 +761,8 @@ protected SplitPredicate predicatesCompensate( queryStructInfo.getPredicates().getPulledUpPredicates(), queryToViewMapping, queryStructInfo, viewStructInfo, cascadesContext); if (!valid) { - queryStructInfo = queryStructInfo.withPredicates( - queryStructInfo.getPredicates().merge(comparisonResult.getQueryAllPulledUpExpressions())); + queryStructInfo = queryStructInfo.withPredicates(queryStructInfo.getPredicates() + .mergePulledUpPredicates(comparisonResult.getQueryAllPulledUpExpressions())); valid = containsNullRejectSlot(requireNoNullableViewSlot, queryStructInfo.getPredicates().getPulledUpPredicates(), queryToViewMapping, queryStructInfo, viewStructInfo, cascadesContext); @@ -770,6 +771,13 @@ protected SplitPredicate predicatesCompensate( return SplitPredicate.INVALID_INSTANCE; } } + // compensate couldNot PulledUp Conjunctions + Map couldNotPulledUpCompensateConjunctions = + Predicates.compensateCouldNotPullUpPredicates(queryStructInfo, viewStructInfo, + viewToQuerySlotMapping, comparisonResult); + if (couldNotPulledUpCompensateConjunctions == null) { + return SplitPredicate.INVALID_INSTANCE; + } // viewEquivalenceClass to query based // equal predicate compensate final Map equalCompensateConjunctions = Predicates.compensateEquivalence( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewScanRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewScanRule.java index 7b2f685cf308c6..1e3fdac4be40a5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewScanRule.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewScanRule.java @@ -81,6 +81,6 @@ protected Plan rewriteQueryByView(MatchMode matchMode, protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext cascadesContext) { PlanCheckContext checkContext = PlanCheckContext.of(ImmutableSet.of()); return structInfo.getTopPlan().accept(StructInfo.SCAN_PLAN_PATTERN_CHECKER, checkContext) - && !checkContext.isContainsTopAggregate(); + && !checkContext.isContainsTopAggregate() && !checkContext.isContainsTopWindow(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewWindowRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewWindowRule.java new file mode 100644 index 00000000000000..9a2913aca433dd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewWindowRule.java @@ -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. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.expressions.Alias; +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 org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; + +import com.google.common.collect.ImmutableMap; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * AbstractMaterializedViewWindowRule + */ +public abstract class AbstractMaterializedViewWindowRule extends AbstractMaterializedViewRule { + + + /** + * compensatePredicates should be pulled up through window + */ + @Override + protected boolean rewriteQueryByViewPreCheck(MatchMode matchMode, StructInfo queryStructInfo, + StructInfo viewStructInfo, SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, + MaterializationContext materializationContext, ComparisonResult comparisonResult) { + boolean superCheck = super.rewriteQueryByViewPreCheck(matchMode, queryStructInfo, viewStructInfo, + viewToQuerySlotMapping, tempRewritedPlan, materializationContext, comparisonResult); + Optional> logicalWindow = + queryStructInfo.getTopPlan().collectFirst(LogicalWindow.class::isInstance); + if (!logicalWindow.isPresent()) { + materializationContext.recordFailReason(queryStructInfo, + "Window rewriteQueryByViewPreCheck fail, logical window is not present", + () -> String.format("expressionToRewritten is %s,\n mvExprToMvScanExprMapping is %s,\n" + + "targetToSourceMapping = %s, tempRewrittenPlan is %s", + queryStructInfo.getExpressions(), materializationContext.getShuttledExprToScanExprMapping(), + viewToQuerySlotMapping, tempRewritedPlan.treeString())); + return false; + } + // if compensatePredicates exists, should be pulled up through window + Set queryCommonPartitionKeySet = logicalWindow.get() + .getCommonPartitionKeyFromWindowExpressions(); + for (Expression conjuct : comparisonResult.getQueryExpressions()) { + if (!queryCommonPartitionKeySet.containsAll(conjuct.getInputSlots())) { + return false; + } + } + return superCheck; + } + + /** + * Rewrite query by view + * + * @param matchMode match mode + * @param queryStructInfo query struct info + * @param viewStructInfo view struct info + * @param viewToQuerySlotMapping slot mapping from view to query + * @param tempRewrittenPlan temporary rewritten plan + * @param materializationContext materialization context + * @param cascadesContext cascades context + * @return rewritten plan + */ + @Override + protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, + SlotMapping viewToQuerySlotMapping, Plan tempRewrittenPlan, MaterializationContext materializationContext, + CascadesContext cascadesContext) { + if (!StructInfo.checkWindowTmpRewrittenPlanIsValid(tempRewrittenPlan)) { + materializationContext.recordFailReason(queryStructInfo, + "Window rewriteQueryByView fail", + () -> String.format("expressionToRewritten is %s,\n mvExprToMvScanExprMapping is %s,\n" + + "targetToSourceMapping = %s, tempRewrittenPlan is %s", + queryStructInfo.getExpressions(), materializationContext.getShuttledExprToScanExprMapping(), + viewToQuerySlotMapping, tempRewrittenPlan.treeString())); + return null; + } + // Rewrite top projects, represent the query projects by view + List expressionsRewritten = rewriteExpression( + queryStructInfo.getExpressions(), + queryStructInfo.getTopPlan(), + materializationContext.getShuttledExprToScanExprMapping(), + viewToQuerySlotMapping, + queryStructInfo.getTableBitSet(), + ImmutableMap.of(), cascadesContext + ); + // Can not rewrite, bail out + if (expressionsRewritten.isEmpty()) { + materializationContext.recordFailReason(queryStructInfo, + "Rewrite expressions by view in window scan fail", + () -> String.format("expressionToRewritten is %s,\n mvExprToMvScanExprMapping is %s,\n" + + "targetToSourceMapping = %s", queryStructInfo.getExpressions(), + materializationContext.getShuttledExprToScanExprMapping(), + viewToQuerySlotMapping)); + return null; + } + return new LogicalProject<>( + expressionsRewritten.stream() + .map(expression -> expression instanceof NamedExpression ? expression : new Alias(expression)) + .map(NamedExpression.class::cast) + .collect(Collectors.toList()), tempRewrittenPlan); + } +} 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 fce8e2f520f0d3..f6d23e6a08a3e2 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 @@ -244,8 +244,19 @@ private boolean tryEliminateNodesAndEdge() { } private boolean compareNodeWithExpr(StructInfoNode query, StructInfoNode view) { - List> queryExprSetList = query.getExprSetList(); - List> viewExprSetList = view.getExprSetList(); + Pair>, List>> queryExprSetList = query.getExprSetList(); + Pair>, List>> viewExprSetList = view.getExprSetList(); + if (queryExprSetList == null || viewExprSetList == null) { + return false; + } + return compareNodeWithExpr(query, ExpressionPosition.NODE_COULD_MOVE, + queryExprSetList.key(), viewExprSetList.key()) + && compareNodeWithExpr(query, ExpressionPosition.NODE_COULD_NOT_MOVE, + queryExprSetList.value(), viewExprSetList.value()); + } + + private boolean compareNodeWithExpr(StructInfoNode query, ExpressionPosition expressionPosition, + List> queryExprSetList, List> viewExprSetList) { if (queryExprSetList == null || viewExprSetList == null || queryExprSetList.size() != viewExprSetList.size()) { return false; @@ -256,8 +267,7 @@ private boolean compareNodeWithExpr(StructInfoNode query, StructInfoNode view) { Set mappingQueryExprSet = new HashSet<>(); for (Expression queryExpression : queryExpressions) { Optional mappingViewExprByQueryExpr = getMappingViewExprByQueryExpr(queryExpression, query, - this.logicalCompatibilityContext, - ExpressionPosition.NODE); + this.logicalCompatibilityContext, expressionPosition); if (!mappingViewExprByQueryExpr.isPresent()) { return false; } @@ -619,9 +629,12 @@ private Optional getMappingViewExprByQueryExpr(Expression queryExpre } else if (ExpressionPosition.FILTER_EDGE.equals(expressionPosition)) { queryShuttledExpr = context.getQueryFilterShuttledExpr(queryExpression); viewExpressions = context.getViewFilterExprFromQuery(queryShuttledExpr); + } else if (ExpressionPosition.NODE_COULD_MOVE.equals(expressionPosition)) { + queryShuttledExpr = context.getQueryNodeShuttledCouldMoveExpr(queryExpression); + viewExpressions = context.getViewNodeCouldMoveExprFromQuery(queryShuttledExpr); } else { - queryShuttledExpr = context.getQueryNodeShuttledExpr(queryExpression); - viewExpressions = context.getViewNodeExprFromQuery(queryShuttledExpr); + queryShuttledExpr = context.getQueryNodeShuttledCouldNotMoveExpr(queryExpression); + viewExpressions = context.getViewNodeCouldNotMoveExprFromQuery(queryShuttledExpr); } if (viewExpressions.size() == 1) { return Optional.of(viewExpressions.iterator().next().key()); 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 77ab37873d06b4..b0438cf59df38a 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 @@ -56,8 +56,12 @@ public class LogicalCompatibilityContext { queryToViewJoinEdgeExpressionMappingSupplier; private final Supplier> queryToQueryShuttledJoinExpressionMappingSupplier; private final Supplier>> - queryToViewNodeExpressionMappingSupplier; - private final Supplier> queryToQueryShuttledNodeExpressionMappingSupplier; + queryToViewNodeCouldMoveExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledNodeCouldMoveExpressionMappingSupplier; + + private final Supplier>> + queryToViewNodeCouldNotMoveExpressionMappingSupplier; + private final Supplier> queryToQueryShuttledNodeCouldNotMoveExpressionMappingSupplier; private final Supplier>> queryToViewFilterEdgeExpressionMappingSupplier; private final Supplier> queryToQueryShuttledFilterExpressionMappingSupplier; @@ -77,13 +81,26 @@ private LogicalCompatibilityContext(BiMap queryT this.queryToQueryShuttledJoinExpressionMappingSupplier = Suppliers.memoize( () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.JOIN_EDGE)); - this.queryToViewNodeExpressionMappingSupplier = + this.queryToViewNodeCouldMoveExpressionMappingSupplier = + Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, + queryStructInfo.getShuttledExpressionsToExpressionsMap() + .get(ExpressionPosition.NODE_COULD_MOVE), + viewStructInfo.getShuttledExpressionsToExpressionsMap() + .get(ExpressionPosition.NODE_COULD_MOVE))); + + this.queryToQueryShuttledNodeCouldMoveExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.NODE_COULD_MOVE)); + + this.queryToViewNodeCouldNotMoveExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, - queryStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE), - viewStructInfo.getShuttledExpressionsToExpressionsMap().get(ExpressionPosition.NODE))); + queryStructInfo.getShuttledExpressionsToExpressionsMap() + .get(ExpressionPosition.NODE_COULD_NOT_MOVE), + viewStructInfo.getShuttledExpressionsToExpressionsMap() + .get(ExpressionPosition.NODE_COULD_NOT_MOVE))); - this.queryToQueryShuttledNodeExpressionMappingSupplier = Suppliers.memoize( - () -> queryStructInfo.getExpressionToShuttledExpressionToMap().get(ExpressionPosition.NODE)); + this.queryToQueryShuttledNodeCouldNotMoveExpressionMappingSupplier = Suppliers.memoize( + () -> queryStructInfo.getExpressionToShuttledExpressionToMap() + .get(ExpressionPosition.NODE_COULD_NOT_MOVE)); this.queryToViewFilterEdgeExpressionMappingSupplier = Suppliers.memoize(() -> generateExpressionMapping(viewToQuerySlotMapping, @@ -125,12 +142,20 @@ public Expression getQueryFilterShuttledExpr(Expression queryFilterExpr) { return queryToQueryShuttledFilterExpressionMappingSupplier.get().get(queryFilterExpr); } - public Collection> getViewNodeExprFromQuery(Expression queryJoinExpr) { - return queryToViewNodeExpressionMappingSupplier.get().get(queryJoinExpr); + public Collection> getViewNodeCouldMoveExprFromQuery(Expression queryJoinExpr) { + return queryToViewNodeCouldMoveExpressionMappingSupplier.get().get(queryJoinExpr); + } + + public Expression getQueryNodeShuttledCouldMoveExpr(Expression queryNodeExpr) { + return queryToQueryShuttledNodeCouldMoveExpressionMappingSupplier.get().get(queryNodeExpr); + } + + public Collection> getViewNodeCouldNotMoveExprFromQuery(Expression queryJoinExpr) { + return queryToViewNodeCouldNotMoveExpressionMappingSupplier.get().get(queryJoinExpr); } - public Expression getQueryNodeShuttledExpr(Expression queryNodeExpr) { - return queryToQueryShuttledNodeExpressionMappingSupplier.get().get(queryNodeExpr); + public Expression getQueryNodeShuttledCouldNotMoveExpr(Expression queryNodeExpr) { + return queryToQueryShuttledNodeCouldNotMoveExpressionMappingSupplier.get().get(queryNodeExpr); } /** @@ -224,8 +249,8 @@ public String toString() { queryToViewJoinEdgeExpressionMappingSupplier.get() == null ? "" : queryToViewJoinEdgeExpressionMappingSupplier.get().toString(), "queryToViewNodeExpressionMapping", - queryToViewNodeExpressionMappingSupplier.get() == null - ? "" : queryToViewNodeExpressionMappingSupplier.get().toString(), + queryToViewNodeCouldMoveExpressionMappingSupplier.get() == null + ? "" : queryToViewNodeCouldMoveExpressionMappingSupplier.get().toString(), "queryToViewFilterEdgeExpressionMapping", queryToViewFilterEdgeExpressionMappingSupplier.get() == null ? "" : queryToViewFilterEdgeExpressionMappingSupplier.get().toString()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowAggregateRule.java new file mode 100644 index 00000000000000..f1b6905e2c3c7b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowAggregateRule.java @@ -0,0 +1,87 @@ +// 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.CascadesContext; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * AbstractMaterializedViewWindowRule + * This is responsible for common window function rewriting + */ +public class MaterializedViewWindowAggregateRule extends AbstractMaterializedViewWindowRule { + + public static MaterializedViewWindowAggregateRule INSTANCE = new MaterializedViewWindowAggregateRule(); + + @Override + protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, + SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext, + CascadesContext cascadesContext) { + return super.rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, viewToQuerySlotMapping, + tempRewritedPlan, materializationContext, cascadesContext); + } + + /** + * Check window pattern is valid or not + */ + @Override + protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext cascadesContext) { + PlanCheckContext checkContext = PlanCheckContext.of(SUPPORTED_JOIN_TYPE_SET); + return structInfo.getTopPlan().accept(StructInfo.PLAN_PATTERN_CHECKER, checkContext) + && checkContext.isContainsTopAggregate() && checkContext.isContainsTopWindow() + && checkContext.getTopAggregateNum() <= 1 && checkContext.getTopWindowNum() <= 1 + && !checkContext.isWindowUnderAggregate(); + } + + @Override + public List buildRules() { + return ImmutableList.of( + logicalWindow(logicalAggregate(any().when(LogicalPlan.class::isInstance))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_WINDOW_AGGREGATE), + logicalProject(logicalWindow(logicalAggregate(any().when(LogicalPlan.class::isInstance)))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_WINDOW_AGGREGATE), + logicalFilter(logicalWindow(logicalAggregate(any().when(LogicalPlan.class::isInstance)))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_WINDOW_AGGREGATE), + logicalProject(logicalFilter(logicalWindow(logicalAggregate( + any().when(LogicalPlan.class::isInstance))))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_AGGREGATE), + logicalFilter(logicalProject(logicalWindow(logicalAggregate( + any().when(LogicalPlan.class::isInstance))))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_AGGREGATE) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowJoinRule.java new file mode 100644 index 00000000000000..f1f05d25cc3343 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowJoinRule.java @@ -0,0 +1,97 @@ +// 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.CascadesContext; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * Materialized view rewrite rule for window on join + */ +public class MaterializedViewWindowJoinRule extends AbstractMaterializedViewWindowRule { + + public static MaterializedViewWindowJoinRule INSTANCE = new MaterializedViewWindowJoinRule(); + + @Override + protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, + SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext, + CascadesContext cascadesContext) { + return super.rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, viewToQuerySlotMapping, + tempRewritedPlan, materializationContext, cascadesContext); + } + + /** + * Check window pattern is valid or not + */ + @Override + protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext cascadesContext) { + PlanCheckContext checkContext = PlanCheckContext.of(SUPPORTED_JOIN_TYPE_SET); + return structInfo.getTopPlan().accept(StructInfo.PLAN_PATTERN_CHECKER, checkContext) + && !checkContext.isContainsTopAggregate() && checkContext.isContainsTopWindow() + && checkContext.getTopWindowNum() <= 1; + } + + @Override + public List buildRules() { + return ImmutableList.of( + logicalFilter(logicalWindow(logicalUnary(logicalJoin(any().when(LogicalPlan.class::isInstance), + any().when(LogicalPlan.class::isInstance))) + .when(node -> node instanceof LogicalProject || node instanceof LogicalFilter))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_WINDOW_JOIN), + logicalProject(logicalWindow(logicalUnary(logicalJoin(any().when(LogicalPlan.class::isInstance), + any().when(LogicalPlan.class::isInstance))) + .when(node -> node instanceof LogicalProject || node instanceof LogicalFilter))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_WINDOW_JOIN), + logicalFilter( + logicalProject(logicalWindow(logicalUnary(logicalJoin(any().when(LogicalPlan.class::isInstance), + any().when(LogicalPlan.class::isInstance))) + .when(node -> node instanceof LogicalProject || node instanceof LogicalFilter)))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_JOIN), + logicalProject( + logicalFilter(logicalWindow(logicalUnary(logicalJoin(any().when(LogicalPlan.class::isInstance), + any().when(LogicalPlan.class::isInstance))) + .when(node -> node instanceof LogicalProject || node instanceof LogicalFilter)))) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_JOIN), + logicalWindow(logicalUnary(logicalJoin(any().when(LogicalPlan.class::isInstance), + any().when(LogicalPlan.class::isInstance))) + .when(node -> node instanceof LogicalProject || node instanceof LogicalFilter)) + .thenApplyMultiNoThrow(ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_WINDOW_JOIN) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowScanRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowScanRule.java new file mode 100644 index 00000000000000..01ffb0ad351053 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewWindowScanRule.java @@ -0,0 +1,96 @@ +// 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.CascadesContext; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.trees.plans.Plan; +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.LogicalProject; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.List; + +/** + * Materialized view rewrite rule for window on scan + */ +public class MaterializedViewWindowScanRule extends AbstractMaterializedViewWindowRule { + + public static MaterializedViewWindowScanRule INSTANCE = new MaterializedViewWindowScanRule(); + + @Override + protected Plan rewriteQueryByView(MatchMode matchMode, StructInfo queryStructInfo, StructInfo viewStructInfo, + SlotMapping viewToQuerySlotMapping, Plan tempRewritedPlan, MaterializationContext materializationContext, + CascadesContext cascadesContext) { + return super.rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, viewToQuerySlotMapping, + tempRewritedPlan, materializationContext, cascadesContext); + } + + /** + * Check window pattern is valid or not + */ + @Override + protected boolean checkQueryPattern(StructInfo structInfo, CascadesContext cascadesContext) { + PlanCheckContext checkContext = PlanCheckContext.of(ImmutableSet.of()); + return structInfo.getTopPlan().accept(StructInfo.SCAN_PLAN_PATTERN_CHECKER, checkContext) + && !checkContext.isContainsTopAggregate() && checkContext.isContainsTopWindow() + && checkContext.getTopWindowNum() <= 1; + } + + @Override + public List buildRules() { + return ImmutableList.of( + logicalFilter(logicalProject(logicalWindow(subTree( + LogicalProject.class, LogicalFilter.class, LogicalCatalogRelation.class)))) + .thenApplyMultiNoThrow( + ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_WINDOW_SCAN), + logicalProject(logicalFilter(logicalWindow(subTree( + LogicalProject.class, LogicalFilter.class, LogicalCatalogRelation.class)))) + .thenApplyMultiNoThrow( + ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_WINDOW_SCAN), + logicalProject(logicalWindow(subTree( + LogicalProject.class, LogicalFilter.class, LogicalCatalogRelation.class))) + .thenApplyMultiNoThrow( + ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_WINDOW_SCAN), + logicalFilter(logicalWindow(subTree( + LogicalProject.class, LogicalFilter.class, LogicalCatalogRelation.class))) + .thenApplyMultiNoThrow( + ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_FILTER_WINDOW_SCAN), + logicalWindow(subTree( + LogicalProject.class, LogicalFilter.class, LogicalCatalogRelation.class)) + .thenApplyMultiNoThrow( + ctx -> { + return rewrite(ctx.root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_WINDOW_SCAN) + ); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java index 18095d5f60ccb4..5ccf624c510d54 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java @@ -37,9 +37,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; 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; @@ -57,23 +59,30 @@ public class Predicates { // Predicates that can be pulled up private final Set pulledUpPredicates; + // Predicates that can not be pulled up, should be equals between query and view + private final Set couldNotPulledUpPredicates; - public Predicates(Set pulledUpPredicates) { + public Predicates(Set pulledUpPredicates, Set couldNotPulledUpPredicates) { this.pulledUpPredicates = pulledUpPredicates; + this.couldNotPulledUpPredicates = couldNotPulledUpPredicates; } - public static Predicates of(Set pulledUpPredicates) { - return new Predicates(pulledUpPredicates); + public static Predicates of(Set pulledUpPredicates, Set predicatesUnderBreaker) { + return new Predicates(pulledUpPredicates, predicatesUnderBreaker); } public Set getPulledUpPredicates() { return pulledUpPredicates; } - public Predicates merge(Collection predicates) { + public Set getCouldNotPulledUpPredicates() { + return couldNotPulledUpPredicates; + } + + public Predicates mergePulledUpPredicates(Collection predicates) { Set mergedPredicates = new HashSet<>(predicates); mergedPredicates.addAll(this.pulledUpPredicates); - return new Predicates(mergedPredicates); + return new Predicates(mergedPredicates, this.couldNotPulledUpPredicates); } /** @@ -84,6 +93,41 @@ public static SplitPredicate splitPredicates(Expression expression) { return predicatesSplit.getSplitPredicate(); } + /** + * try to compensate could not pull up predicates + */ + public static Map compensateCouldNotPullUpPredicates( + StructInfo queryStructInfo, StructInfo viewStructInfo, + SlotMapping viewToQuerySlotMapping, ComparisonResult comparisonResult) { + + Predicates queryStructInfoPredicates = queryStructInfo.getPredicates(); + Predicates viewStructInfoPredicates = viewStructInfo.getPredicates(); + if (queryStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty() + && viewStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty()) { + return ImmutableMap.of(); + } + if (queryStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty() + && !viewStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty()) { + return null; + } + if (!queryStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty() + && viewStructInfoPredicates.getCouldNotPulledUpPredicates().isEmpty()) { + return null; + } + + List viewPredicatesShuttled = ExpressionUtils.shuttleExpressionWithLineage( + Lists.newArrayList(viewStructInfoPredicates.getCouldNotPulledUpPredicates()), + viewStructInfo.getTopPlan(), new BitSet()); + List viewPredicatesQueryBased = ExpressionUtils.replace((List) viewPredicatesShuttled, + viewToQuerySlotMapping.toSlotReferenceMap()); + // could not be pulled up predicates in query and view should be same + if (queryStructInfoPredicates.getCouldNotPulledUpPredicates().equals( + Sets.newHashSet(viewPredicatesQueryBased))) { + return ImmutableMap.of(); + } + return null; + } + /** * compensate equivalence predicates */ @@ -250,7 +294,8 @@ public static Map compensateResidualPredicate(Struct @Override public String toString() { - return Utils.toSqlString("Predicates", "pulledUpPredicates", pulledUpPredicates); + return Utils.toSqlString("Predicates", "pulledUpPredicates", pulledUpPredicates, + "predicatesUnderBreaker", couldNotPulledUpPredicates); } /** 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 7b8b976295d1c9..ed232041d4a065 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 @@ -28,6 +28,7 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils.TableQueryOperatorChecker; import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; +import org.apache.doris.nereids.rules.rewrite.PushDownFilterThroughWindow; import org.apache.doris.nereids.trees.copier.DeepCopierContext; import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; import org.apache.doris.nereids.trees.expressions.EqualTo; @@ -53,7 +54,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +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.util.ExpressionUtils; @@ -65,6 +68,7 @@ import java.util.ArrayList; import java.util.BitSet; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; @@ -83,7 +87,7 @@ public class StructInfo { public static final ScanPlanPatternChecker SCAN_PLAN_PATTERN_CHECKER = new ScanPlanPatternChecker(); // struct info splitter public static final PlanSplitter PLAN_SPLITTER = new PlanSplitter(); - private static final PredicateCollector PREDICATE_COLLECTOR = new PredicateCollector(); + public static final PredicateCollector PREDICATE_COLLECTOR = new PredicateCollector(); // source data private final Plan originalPlan; private final ObjectId originalPlanId; @@ -185,15 +189,26 @@ private static boolean collectStructInfoFromGraph(HyperGraph hyperGraph, relations.addAll(structInfoNode.getCatalogRelation()); nodeRelations.forEach(relation -> hyperTableBitSet.set(relation.getRelationId().asInt())); // record expressions in node - List nodeExpressions = structInfoNode.getExpressions(); + List nodeExpressions = structInfoNode.getCouldMoveExpressions(); if (nodeExpressions != null) { List shuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( nodeExpressions, structInfoNode.getPlan(), new BitSet()); for (int index = 0; index < nodeExpressions.size(); index++) { putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, - expressionToShuttledExpressionToMap, - ExpressionPosition.NODE, shuttledExpressions.get(index), nodeExpressions.get(index), node); + expressionToShuttledExpressionToMap, ExpressionPosition.NODE_COULD_MOVE, + shuttledExpressions.get(index), nodeExpressions.get(index), node); + } + } + nodeExpressions = structInfoNode.getCouldNotMoveExpressions(); + if (nodeExpressions != null) { + List shuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage( + nodeExpressions, structInfoNode.getPlan(), + new BitSet()); + for (int index = 0; index < nodeExpressions.size(); index++) { + putShuttledExpressionToExpressionsMap(shuttledExpressionsToExpressionsMap, + expressionToShuttledExpressionToMap, ExpressionPosition.NODE_COULD_NOT_MOVE, + shuttledExpressions.get(index), nodeExpressions.get(index), node); } } // every node should only have one relation, this is for LogicalCompatibilityContext @@ -318,9 +333,10 @@ public static StructInfo of(Plan originalPlan, @Nullable Plan topPlan, @Nullable ((AbstractPlan) relation).accept(TableQueryOperatorChecker.INSTANCE, null)); valid = valid && !invalid; // collect predicate from top plan which not in hyper graph - Set topPlanPredicates = new LinkedHashSet<>(); - topPlan.accept(PREDICATE_COLLECTOR, topPlanPredicates); - Predicates predicates = Predicates.of(topPlanPredicates); + PredicateCollectorContext predicateCollectorContext = new PredicateCollectorContext(); + topPlan.accept(PREDICATE_COLLECTOR, predicateCollectorContext); + Predicates predicates = Predicates.of(predicateCollectorContext.getCouldPullUpPredicates(), + predicateCollectorContext.getCouldNotPullUpPredicates()); // this should use the output of originalPlan to make sure the output right order List planOutputShuttledExpressions = ExpressionUtils.shuttleExpressionWithLineage(originalPlan.getOutput(), originalPlan, new BitSet()); @@ -452,21 +468,75 @@ public String toString() { return "StructInfo{ originalPlanId = " + originalPlanId + ", relations = " + relations + '}'; } - private static class PredicateCollector extends DefaultPlanVisitor> { + private static class RelationCollector extends DefaultPlanVisitor> { @Override - public Void visit(Plan plan, Set predicates) { - // Just collect the filter in top plan, if meet other node except project and filter, return - if (!(plan instanceof LogicalProject) + public Void visit(Plan plan, List collectedRelations) { + if (plan instanceof CatalogRelation) { + collectedRelations.add((CatalogRelation) plan); + } + return super.visit(plan, collectedRelations); + } + } + + private static class PredicateCollector extends DefaultPlanVisitor { + @Override + public Void visit(Plan plan, PredicateCollectorContext collectorContext) { + // Just collect the filter in top plan, if meet other node except the following node, return + if (!(plan instanceof LogicalSink) + && !(plan instanceof LogicalProject) && !(plan instanceof LogicalFilter) && !(plan instanceof LogicalAggregate) + && !(plan instanceof LogicalWindow) && !(plan instanceof LogicalSort) && !(plan instanceof LogicalRepeat)) { return null; } - if (plan instanceof LogicalFilter) { - predicates.addAll(ExpressionUtils.extractConjunction(((LogicalFilter) plan).getPredicate())); + return super.visit(plan, collectorContext); + } + + @Override + public Void visitLogicalFilter(LogicalFilter filter, PredicateCollectorContext context) { + if (context.getWindowCommonPartitionKeys().isEmpty()) { + context.getCouldPullUpPredicates().addAll(filter.getConjuncts()); + } else { + // if the filter contains the partition key of window, it can be pulled up + for (Expression conjunct : filter.getConjuncts()) { + if (PushDownFilterThroughWindow.canPushDown(conjunct, context.getWindowCommonPartitionKeys())) { + context.getCouldPullUpPredicates().add(conjunct); + } else { + context.getCouldNotPullUpPredicates().add(conjunct); + } + } } - return super.visit(plan, predicates); + return super.visit(filter, context); + } + + @Override + public Void visitLogicalWindow(LogicalWindow window, PredicateCollectorContext context) { + Set commonPartitionKeys = window.getCommonPartitionKeyFromWindowExpressions(); + context.getWindowCommonPartitionKeys().addAll(commonPartitionKeys); + return super.visit(window, context); + } + } + + /** + * PredicateCollectorContext + */ + public static class PredicateCollectorContext { + private Set couldPullUpPredicates = new HashSet<>(); + private Set couldNotPullUpPredicates = new HashSet<>(); + private Set windowCommonPartitionKeys = new HashSet<>(); + + public Set getCouldPullUpPredicates() { + return couldPullUpPredicates; + } + + public Set getCouldNotPullUpPredicates() { + return couldNotPullUpPredicates; + } + + public Set getWindowCommonPartitionKeys() { + return windowCommonPartitionKeys; } } @@ -553,10 +623,19 @@ public boolean isBoundary(Plan plan) { * The context for plan check context, make sure that the plan in query and mv is valid or not */ public static class PlanCheckContext { - // the aggregate above join + // Record if aggregate is above join or not private boolean containsTopAggregate = false; private int topAggregateNum = 0; + // This indicates whether the operators above the join contain any window operator. + private boolean containsTopWindow = false; + // This records the number of window operators above the join. + private int topWindowNum = 0; private boolean alreadyMeetJoin = false; + // Records whether an Aggregate operator has been meet when check window operator + private boolean alreadyMeetAggregate = false; + // Indicates if a window operator is under an Aggregate operator, because the window operator under + // aggregate is not supported now. + private boolean windowUnderAggregate = false; private final Set supportJoinTypes; public PlanCheckContext(Set supportJoinTypes) { @@ -571,6 +650,22 @@ public void setContainsTopAggregate(boolean containsTopAggregate) { this.containsTopAggregate = containsTopAggregate; } + public boolean isContainsTopWindow() { + return containsTopWindow; + } + + public void setContainsTopWindow(boolean containsTopWindow) { + this.containsTopWindow = containsTopWindow; + } + + public int getTopWindowNum() { + return topWindowNum; + } + + public void plusTopWindowNum() { + this.topWindowNum += 1; + } + public boolean isAlreadyMeetJoin() { return alreadyMeetJoin; } @@ -591,6 +686,22 @@ public void plusTopAggregateNum() { this.topAggregateNum += 1; } + public boolean isAlreadyMeetAggregate() { + return alreadyMeetAggregate; + } + + public void setAlreadyMeetAggregate(boolean alreadyMeetAggregate) { + this.alreadyMeetAggregate = alreadyMeetAggregate; + } + + public boolean isWindowUnderAggregate() { + return windowUnderAggregate; + } + + public void setWindowUnderAggregate(boolean windowUnderAggregate) { + this.windowUnderAggregate = windowUnderAggregate; + } + public static PlanCheckContext of(Set supportJoinTypes) { return new PlanCheckContext(supportJoinTypes); } @@ -615,11 +726,25 @@ public Boolean visitLogicalAggregate(LogicalAggregate aggregate, PlanCheckContext checkContext) { if (!checkContext.isAlreadyMeetJoin()) { checkContext.setContainsTopAggregate(true); + checkContext.setAlreadyMeetAggregate(true); checkContext.plusTopAggregateNum(); } return visit(aggregate, checkContext); } + @Override + public Boolean visitLogicalWindow(LogicalWindow window, PlanCheckContext checkContext) { + if (!checkContext.isAlreadyMeetJoin()) { + if (checkContext.isAlreadyMeetAggregate()) { + checkContext.setWindowUnderAggregate(true); + return false; + } + checkContext.setContainsTopWindow(true); + checkContext.plusTopWindowNum(); + } + return visit(window, checkContext); + } + @Override public Boolean visitGroupPlan(GroupPlan groupPlan, PlanCheckContext checkContext) { return groupPlan.getGroup().getLogicalExpressions().stream() @@ -635,7 +760,8 @@ public Boolean visit(Plan plan, PlanCheckContext checkContext) { || plan instanceof LogicalSort || plan instanceof LogicalAggregate || plan instanceof GroupPlan - || plan instanceof LogicalRepeat) { + || plan instanceof LogicalRepeat + || plan instanceof LogicalWindow) { return doVisit(plan, checkContext); } return false; @@ -663,13 +789,21 @@ public Boolean visitGroupPlan(GroupPlan groupPlan, PlanCheckContext checkContext .anyMatch(logicalExpression -> logicalExpression.getPlan().accept(this, checkContext)); } + @Override + public Boolean visitLogicalWindow(LogicalWindow window, PlanCheckContext checkContext) { + checkContext.setContainsTopWindow(true); + checkContext.plusTopWindowNum(); + return visit(window, checkContext); + } + @Override public Boolean visit(Plan plan, PlanCheckContext checkContext) { if (plan instanceof Filter || plan instanceof Project || plan instanceof CatalogRelation || plan instanceof GroupPlan - || plan instanceof LogicalRepeat) { + || plan instanceof LogicalRepeat + || plan instanceof LogicalWindow) { return doVisit(plan, checkContext); } return false; @@ -745,7 +879,34 @@ public static Pair addFilterOnTableScan(Plan queryPlan, Map() { + @Override + public Boolean visit(Plan plan, Void context) { + if (plan instanceof LogicalProject || plan instanceof CatalogRelation + || plan instanceof LogicalFilter) { + boolean isValid; + for (Plan child : plan.children()) { + isValid = child.accept(this, context); + if (!isValid) { + return false; + } + } + return true; + } + return false; + } + }, null); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughWindow.java index a949cbc9453955..678e19da169732 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughWindow.java @@ -64,7 +64,7 @@ public Rule build() { Set bottomConjuncts = Sets.newHashSet(); Set upperConjuncts = Sets.newHashSet(); for (Expression expr : filter.getConjuncts()) { - if (commonPartitionKeys.containsAll(expr.getInputSlots())) { + if (canPushDown(expr, commonPartitionKeys)) { bottomConjuncts.add(expr); } else { upperConjuncts.add(expr); @@ -86,4 +86,7 @@ public Rule build() { }).toRule(RuleType.PUSH_DOWN_FILTER_THROUGH_WINDOW); } + public static boolean canPushDown(Expression conjunct, Set commonPartitionKeys) { + return commonPartitionKeys.containsAll(conjunct.getInputSlots()); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java index fde80a4d5b67ed..72067d73ef8611 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWindow.java @@ -50,6 +50,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import java.util.ArrayList; import java.util.List; import java.util.Objects; import java.util.Optional; @@ -97,6 +98,14 @@ public List getExpressions() { return windowExpressions; } + public List getActualWindowExpressions() { + List actualWindowExpressions = new ArrayList<>(); + for (NamedExpression expr : windowExpressions) { + actualWindowExpressions.add((WindowExpression) (expr.child(0))); + } + return actualWindowExpressions; + } + public LogicalWindow withExpressionsAndChild(List windowExpressions, Plan child) { return new LogicalWindow<>(windowExpressions, isChecked, child); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PredicatesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PredicatesTest.java new file mode 100644 index 00000000000000..6b5e05962051e9 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/PredicatesTest.java @@ -0,0 +1,190 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.mv; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.rules.exploration.mv.ComparisonResult; +import org.apache.doris.nereids.rules.exploration.mv.HyperGraphComparator; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewUtils; +import org.apache.doris.nereids.rules.exploration.mv.Predicates; +import org.apache.doris.nereids.rules.exploration.mv.Predicates.ExpressionInfo; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo; +import org.apache.doris.nereids.rules.exploration.mv.mapping.RelationMapping; +import org.apache.doris.nereids.rules.exploration.mv.mapping.SlotMapping; +import org.apache.doris.nereids.sqltest.SqlTestBase; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.util.PlanChecker; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.BitSet; +import java.util.Map; + +/** Test the method in Predicates*/ +public class PredicatesTest extends SqlTestBase { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("predicates_test"); + useDatabase("predicates_test"); + + createTables( + "CREATE TABLE IF NOT EXISTS T1 (\n" + + " id bigint,\n" + + " score bigint\n" + + ")\n" + + "DUPLICATE KEY(id)\n" + + "DISTRIBUTED BY HASH(id, score) BUCKETS 10\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\", \n" + + " \"colocate_with\" = \"T0\"\n" + + ")\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" + + ",ELIMINATE_GROUP_BY_KEY_BY_UNIFORM" + + ",ELIMINATE_CONST_JOIN_CONDITION" + + ",CONSTANT_PROPAGATION" + + ",INFER_PREDICATES" + ); + } + + @Test + public void testCompensateCouldNotPullUpPredicatesFail() { + CascadesContext mvContext = createCascadesContext( + "select \n" + + "id,\n" + + "FIRST_VALUE(id) OVER (\n" + + " PARTITION BY score \n" + + " ORDER BY score NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value\n" + + "from \n" + + "T1\n" + + "where score > 10 and id < 5;", + connectContext + ); + Plan mvPlan = PlanChecker.from(mvContext) + .analyze() + .rewrite() + .getPlan().child(0); + + CascadesContext queryContext = createCascadesContext( + "select \n" + + "id,\n" + + "FIRST_VALUE(id) OVER (\n" + + " PARTITION BY score \n" + + " ORDER BY score NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value\n" + + "from \n" + + "T1\n" + + "where score > 10 and id < 1;", + connectContext + ); + Plan queryPlan = PlanChecker.from(queryContext) + .analyze() + .rewrite() + .getAllPlan().get(0).child(0); + + StructInfo mvStructInfo = MaterializedViewUtils.extractStructInfo(mvPlan, mvPlan, + mvContext, new BitSet()).get(0); + StructInfo queryStructInfo = MaterializedViewUtils.extractStructInfo(queryPlan, queryPlan, + queryContext, new BitSet()).get(0); + RelationMapping relationMapping = RelationMapping.generate(mvStructInfo.getRelations(), + queryStructInfo.getRelations(), 16).get(0); + + SlotMapping mvToQuerySlotMapping = SlotMapping.generate(relationMapping); + ComparisonResult comparisonResult = HyperGraphComparator.isLogicCompatible( + queryStructInfo.getHyperGraph(), + mvStructInfo.getHyperGraph(), + constructContext(queryPlan, mvPlan, queryContext)); + + Map expressionExpressionInfoMap = Predicates.compensateCouldNotPullUpPredicates( + queryStructInfo, mvStructInfo, mvToQuerySlotMapping, comparisonResult); + Assertions.assertNull(expressionExpressionInfoMap); + } + + @Test + public void testCompensateCouldNotPullUpPredicatesSuccess() { + CascadesContext mvContext = createCascadesContext( + "select \n" + + "id,\n" + + "FIRST_VALUE(id) OVER (\n" + + " PARTITION BY score \n" + + " ORDER BY score NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value\n" + + "from \n" + + "T1\n" + + "where score > 10 and id < 5;", + connectContext + ); + Plan mvPlan = PlanChecker.from(mvContext) + .analyze() + .rewrite() + .getPlan().child(0); + + CascadesContext queryContext = createCascadesContext( + "select \n" + + "id,\n" + + "FIRST_VALUE(id) OVER (\n" + + " PARTITION BY score \n" + + " ORDER BY score NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value\n" + + "from \n" + + "T1\n" + + "where score > 15 and id < 5;", + connectContext + ); + Plan queryPlan = PlanChecker.from(queryContext) + .analyze() + .rewrite() + .getAllPlan().get(0).child(0); + + StructInfo mvStructInfo = MaterializedViewUtils.extractStructInfo(mvPlan, mvPlan, + mvContext, new BitSet()).get(0); + StructInfo queryStructInfo = MaterializedViewUtils.extractStructInfo(queryPlan, queryPlan, + queryContext, new BitSet()).get(0); + RelationMapping relationMapping = RelationMapping.generate(mvStructInfo.getRelations(), + queryStructInfo.getRelations(), 16).get(0); + + SlotMapping mvToQuerySlotMapping = SlotMapping.generate(relationMapping); + ComparisonResult comparisonResult = HyperGraphComparator.isLogicCompatible( + queryStructInfo.getHyperGraph(), + mvStructInfo.getHyperGraph(), + constructContext(queryPlan, mvPlan, queryContext)); + + Map compensateCouldNotPullUpPredicates = Predicates.compensateCouldNotPullUpPredicates( + queryStructInfo, mvStructInfo, mvToQuerySlotMapping, comparisonResult); + Assertions.assertNotNull(compensateCouldNotPullUpPredicates); + Assertions.assertTrue(compensateCouldNotPullUpPredicates.isEmpty()); + + Map compensateRangePredicates = Predicates.compensateRangePredicate( + queryStructInfo, mvStructInfo, mvToQuerySlotMapping, comparisonResult, + queryContext); + Assertions.assertNotNull(compensateRangePredicates); + Assertions.assertEquals(1, compensateRangePredicates.size()); + } +} diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/StructInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/StructInfoTest.java new file mode 100644 index 00000000000000..3e720b93f4cb23 --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/mv/StructInfoTest.java @@ -0,0 +1,242 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.mv; + +import org.apache.doris.nereids.rules.exploration.mv.AbstractMaterializedViewJoinRule; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PlanCheckContext; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo.PredicateCollectorContext; +import org.apache.doris.nereids.sqltest.SqlTestBase; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.util.PlanChecker; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +/** + * Test the methods in StructInfo. + */ +public class StructInfoTest extends SqlTestBase { + + @Override + protected void runBeforeAll() throws Exception { + createDatabase("struct_info_test"); + useDatabase("struct_info_test"); + createTables("CREATE TABLE IF NOT EXISTS orders_arr (\n" + + " o_orderkey INTEGER NOT NULL,\n" + + " o_orderstatus CHAR(1) NOT NULL,\n" + + " o_totalprice DECIMALV3(15,2) NOT NULL,\n" + + " o_custkey INTEGER NOT NULL,\n" + + " o_orderdate DATE NOT NULL,\n" + + " o_orderpriority CHAR(15) NOT NULL, \n" + + " o_shippriority INTEGER NOT NULL,\n" + + " o_comment VARCHAR(79) NOT NULL,\n" + + " o_array1 ARRAY NULL,\n" + + " o_array2 ARRAY NULL\n" + + ")\n" + + "DUPLICATE KEY(o_orderkey, o_orderstatus)\n" + + "DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3\n" + + "PROPERTIES (\n" + + " \"replication_num\" = \"1\"\n" + + ");"); + // Should not make scan to empty relation when the table used by materialized view has no data + connectContext.getSessionVariable().setDisableNereidsRules( + "OLAP_SCAN_PARTITION_PRUNE" + + ",PRUNE_EMPTY_PARTITION" + + ",ELIMINATE_GROUP_BY_KEY_BY_UNIFORM" + + ",ELIMINATE_CONST_JOIN_CONDITION" + + ",CONSTANT_PROPAGATION" + ); + } + + @Test + public void testPlanPatternCheckerWindowAboveAgg() { + PlanChecker.from(connectContext) + .checkExplain("select \n" + + "o_orderkey,\n" + + "FIRST_VALUE(o_custkey) OVER (\n" + + " PARTITION BY o_orderdate \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value,\n" + + "RANK() OVER (\n" + + " PARTITION BY o_orderdate, o_orderstatus \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS rank_value,\n" + + "count(*) \n" + + "from \n" + + "orders_arr\n" + + "group by o_orderkey,\n" + + "o_custkey,\n" + + "o_orderdate,\n" + + "o_orderstatus,\n" + + "o_totalprice;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + PlanCheckContext planCheckContext = new PlanCheckContext( + AbstractMaterializedViewJoinRule.SUPPORTED_JOIN_TYPE_SET); + Boolean valid = rewrittenPlan.child(0).accept( + StructInfo.PLAN_PATTERN_CHECKER, planCheckContext); + Assertions.assertTrue(valid); + Assertions.assertEquals(1, planCheckContext.getTopWindowNum()); + Assertions.assertTrue(planCheckContext.isContainsTopWindow()); + Assertions.assertFalse(planCheckContext.isWindowUnderAggregate()); + }); + } + + @Test + public void testPlanPatternCheckerWindowUnderAgg() { + PlanChecker.from(connectContext) + .checkExplain("select o_orderkey, first_value, count(*)\n" + + "from\n" + + "(\n" + + "select \n" + + "o_orderkey, o_orderdate,\n" + + "FIRST_VALUE(o_custkey) OVER (\n" + + " PARTITION BY o_orderdate \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value,\n" + + "RANK() OVER (\n" + + " PARTITION BY o_orderdate, o_orderstatus \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS rank_value\n" + + "from \n" + + "orders_arr\n" + + ") t\n" + + "group by o_orderkey, first_value;", + nereidsPlanner -> { + + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan(); + PlanCheckContext planCheckContext = new PlanCheckContext( + AbstractMaterializedViewJoinRule.SUPPORTED_JOIN_TYPE_SET); + Boolean valid = rewrittenPlan.child(0).accept( + StructInfo.PLAN_PATTERN_CHECKER, planCheckContext); + Assertions.assertFalse(valid); + Assertions.assertEquals(0, planCheckContext.getTopWindowNum()); + Assertions.assertFalse(planCheckContext.isContainsTopWindow()); + Assertions.assertTrue(planCheckContext.isWindowUnderAggregate()); + }); + } + + @Test + public void testCheckWindowTmpRewrittenPlanInValid() { + PlanChecker.from(connectContext) + .checkExplain("select o_orderkey, c2\n" + + "from (select o_orderkey from orders_arr limit 1) orders_a\n" + + "LATERAL VIEW explode_numbers(0) t1 as c2\n" + + "order by c2;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan().child(0); + Assertions.assertFalse(StructInfo.checkWindowTmpRewrittenPlanIsValid(rewrittenPlan)); + }); + } + + @Test + public void testCheckWindowTmpRewrittenPlanIsValid() { + PlanChecker.from(connectContext) + .checkExplain("select o_orderkey from orders_arr where o_orderkey > 1", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan().child(0); + Assertions.assertTrue(StructInfo.checkWindowTmpRewrittenPlanIsValid(rewrittenPlan)); + }); + } + + @Test + public void testPredicateCollectorWithCouldPullUp() { + PlanChecker.from(connectContext) + .checkExplain("select \n" + + "o_orderkey, o_orderdate,\n" + + "FIRST_VALUE(o_custkey) OVER (\n" + + " PARTITION BY o_orderdate \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value\n" + + "from \n" + + "(\n" + + "select * from orders_arr where o_orderdate > '2025-01-01' and o_custkey = 1\n" + + ") t;", + nereidsPlanner -> { + + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan().child(0); + PredicateCollectorContext predicateCollectorContext = new PredicateCollectorContext(); + rewrittenPlan.accept(StructInfo.PREDICATE_COLLECTOR, predicateCollectorContext); + Assertions.assertEquals(1, + predicateCollectorContext.getCouldPullUpPredicates().size()); + Assertions.assertTrue( + predicateCollectorContext.getCouldPullUpPredicates().stream().allMatch( + expr -> expr.collectFirst( + node -> node instanceof SlotReference + && ((SlotReference) node).getName() + .equalsIgnoreCase("o_orderdate")) + .isPresent())); + Assertions.assertEquals(1, + predicateCollectorContext.getCouldNotPullUpPredicates().size()); + Assertions.assertTrue( + predicateCollectorContext.getCouldNotPullUpPredicates().stream().allMatch( + expr -> expr.collectFirst( + node -> node instanceof SlotReference + && ((SlotReference) node).getName() + .equalsIgnoreCase("o_custkey")) + .isPresent())); + }); + } + + @Test + public void testPredicateCollectorWithCouldNotPullUp() { + PlanChecker.from(connectContext) + .checkExplain("select o_orderkey, first_value\n" + + "from\n" + + "(\n" + + "select \n" + + "o_orderkey, o_orderdate,\n" + + "FIRST_VALUE(o_custkey) OVER (\n" + + " PARTITION BY o_orderdate \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS first_value,\n" + + "RANK() OVER (\n" + + " PARTITION BY o_orderdate, o_orderstatus \n" + + " ORDER BY o_totalprice NULLS LAST\n" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\n" + + " ) AS rank_value\n" + + "from \n" + + "orders_arr\n" + + ") t\n" + + "where first_value > 0;", + nereidsPlanner -> { + Plan rewrittenPlan = nereidsPlanner.getRewrittenPlan().child(0); + PredicateCollectorContext predicateCollectorContext = new PredicateCollectorContext(); + rewrittenPlan.accept(StructInfo.PREDICATE_COLLECTOR, predicateCollectorContext); + Assertions.assertEquals(1, + predicateCollectorContext.getCouldPullUpPredicates().size()); + Assertions.assertEquals(0, + predicateCollectorContext.getCouldNotPullUpPredicates().size()); + Assertions.assertTrue( + predicateCollectorContext.getCouldPullUpPredicates().stream().allMatch( + expr -> expr.collectFirst( + node -> node instanceof SlotReference + && ((SlotReference) node).getName() + .equalsIgnoreCase("first_value")) + .isPresent())); + }); + } +} diff --git a/regression-test/data/nereids_rules_p0/mv/window/window_above_aggregate.out b/regression-test/data/nereids_rules_p0/mv/window/window_above_aggregate.out new file mode 100644 index 00000000000000..a4b4db2fecfb91 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/window/window_above_aggregate.out @@ -0,0 +1,271 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query1_0_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query1_1_before -- + +-- !query1_1_after -- + +-- !query2_0_before -- +3 1 6.50 1 1 0.00 +3 1 6.50 1 2 6.50 +4 2 6.50 2 1 0.00 +5 2 8.50 2 1 0.00 +5 2 8.50 2 2 8.50 + +-- !query2_0_after -- +3 1 6.50 1 1 0.00 +3 1 6.50 1 2 6.50 +4 2 6.50 2 1 0.00 +5 2 8.50 2 1 0.00 +5 2 8.50 2 2 8.50 + +-- !query2_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query2_1_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query2_2_before -- +3 1 6.50 1 1 0.00 3 +3 1 6.50 1 2 6.50 3 +4 2 6.50 2 1 0.00 \N +5 2 8.50 2 1 0.00 3 +5 2 8.50 2 2 8.50 3 + +-- !query2_2_after -- +3 1 6.50 1 1 0.00 3 +3 1 6.50 1 2 6.50 3 +4 2 6.50 2 1 0.00 \N +5 2 8.50 2 1 0.00 3 +5 2 8.50 2 2 8.50 3 + +-- !query2_3_before -- +2 1 6.50 1 1 0.00 \N +3 1 6.50 1 1 0.00 3 +3 1 6.50 1 2 6.50 3 +4 2 6.50 2 1 0.00 \N +5 2 8.50 2 1 0.00 3 +5 2 8.50 2 2 8.50 3 + +-- !query2_3_after -- +2 1 6.50 1 1 0.00 \N +3 1 6.50 1 1 0.00 3 +3 1 6.50 1 2 6.50 3 +4 2 6.50 2 1 0.00 \N +5 2 8.50 2 1 0.00 3 +5 2 8.50 2 2 8.50 3 + +-- !query3_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query3_0_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query3_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query3_1after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query4_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query4_0after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query4_1_before -- +6.50 2023-12-08 1 10.50 o 6.50 2023-12-08 5.50 1 1 2 6.50 +6.50 2023-12-08 1 9.50 o 6.50 2023-12-08 5.50 1 1 1 0.00 +6.50 2023-12-09 1 11.50 o 6.50 2023-12-09 5.50 2 1 1 0.00 +6.50 2023-12-09 1 11.50 o 6.50 2023-12-09 5.50 2 1 1 6.50 +6.50 2023-12-10 1 12.50 o 6.50 2023-12-10 5.50 3 1 1 0.00 +6.50 2023-12-10 1 12.50 o 6.50 2023-12-10 5.50 3 1 1 6.50 +6.50 2023-12-10 1 33.50 o 6.50 2023-12-10 5.50 3 1 4 6.50 +6.50 2023-12-11 2 43.20 o 6.50 2023-12-11 5.50 4 2 1 0.00 +6.50 2023-12-11 2 43.20 o 6.50 2023-12-11 5.50 4 2 1 6.50 +8.50 2023-12-12 2 1.20 o 8.50 2023-12-12 7.50 5 2 1 0.00 +8.50 2023-12-12 2 56.20 o 8.50 2023-12-12 7.50 5 2 2 8.50 + +-- !query4_1after -- +6.50 2023-12-08 1 10.50 o 6.50 2023-12-08 5.50 1 1 2 6.50 +6.50 2023-12-08 1 9.50 o 6.50 2023-12-08 5.50 1 1 1 0.00 +6.50 2023-12-09 1 11.50 o 6.50 2023-12-09 5.50 2 1 1 0.00 +6.50 2023-12-09 1 11.50 o 6.50 2023-12-09 5.50 2 1 1 6.50 +6.50 2023-12-10 1 12.50 o 6.50 2023-12-10 5.50 3 1 1 0.00 +6.50 2023-12-10 1 12.50 o 6.50 2023-12-10 5.50 3 1 1 6.50 +6.50 2023-12-10 1 33.50 o 6.50 2023-12-10 5.50 3 1 4 6.50 +6.50 2023-12-11 2 43.20 o 6.50 2023-12-11 5.50 4 2 1 0.00 +6.50 2023-12-11 2 43.20 o 6.50 2023-12-11 5.50 4 2 1 6.50 +8.50 2023-12-12 2 1.20 o 8.50 2023-12-12 7.50 5 2 1 0.00 +8.50 2023-12-12 2 56.20 o 8.50 2023-12-12 7.50 5 2 2 8.50 + +-- !query5_0_before -- + +-- !query5_0_after -- + +-- !query5_1_before -- + +-- !query5_1_after -- + +-- !query6_0_before -- +3 1 6.50 1 1 0.00 +3 1 6.50 1 2 6.50 +4 2 6.50 2 1 0.00 +5 2 8.50 2 1 0.00 +5 2 8.50 2 2 8.50 + +-- !query6_0_after -- +3 1 6.50 1 1 0.00 +3 1 6.50 1 2 6.50 +4 2 6.50 2 1 0.00 +5 2 8.50 2 1 0.00 +5 2 8.50 2 2 8.50 + +-- !query6_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query6_1_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_0_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_1after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_2_before -- +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query7_2after -- +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query8_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query8_0after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query8_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query8_1after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query9_0_before -- + +-- !query9_0_after -- + +-- !query9_1_before -- + +-- !query9_1_after -- + +-- !query9_2_before -- + +-- !query9_2_after -- + +-- !query10_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query10_0_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query10_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query10_1_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query11_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query11_0_after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query11_1_before -- + +-- !query11_1after -- + +-- !query12_0_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query12_0after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query12_1_before -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + +-- !query12_1after -- +4 2023-12-11 2 6.50 2 1 0.00 +5 2023-12-12 2 8.50 2 1 0.00 +5 2023-12-12 2 8.50 2 2 8.50 + diff --git a/regression-test/data/nereids_rules_p0/mv/window/window_above_join.out b/regression-test/data/nereids_rules_p0/mv/window/window_above_join.out new file mode 100644 index 00000000000000..427fac89370c26 --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/window/window_above_join.out @@ -0,0 +1,453 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query1_0_after -- +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query1_1_before -- + +-- !query1_1_after -- + +-- !query2_0_before -- +3 1 1 0.00 +3 1 1 6.50 +3 1 1 6.50 +3 1 4 6.50 +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query2_0_after -- +3 1 1 0.00 +3 1 1 6.50 +3 1 1 6.50 +3 1 4 6.50 +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query2_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query2_1_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query2_2_before -- +3 1 1 0.00 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 7 6.50 3 +3 1 7 6.50 3 +4 2 1 0.00 \N +4 2 1 6.50 \N +4 2 1 6.50 \N +5 2 1 0.00 3 +5 2 1 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 + +-- !query2_2_after -- +3 1 1 0.00 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 7 6.50 3 +3 1 7 6.50 3 +4 2 1 0.00 \N +4 2 1 6.50 \N +4 2 1 6.50 \N +5 2 1 0.00 3 +5 2 1 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 + +-- !query2_3_before -- +2 1 1 0.00 \N +2 1 1 6.50 \N +2 1 1 6.50 \N +3 1 1 0.00 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 7 6.50 3 +3 1 7 6.50 3 +4 2 1 0.00 \N +4 2 1 6.50 \N +4 2 1 6.50 \N +5 2 1 0.00 3 +5 2 1 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 + +-- !query2_3_after -- +2 1 1 0.00 \N +2 1 1 6.50 \N +2 1 1 6.50 \N +3 1 1 0.00 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 1 6.50 3 +3 1 7 6.50 3 +3 1 7 6.50 3 +4 2 1 0.00 \N +4 2 1 6.50 \N +4 2 1 6.50 \N +5 2 1 0.00 3 +5 2 1 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 +5 2 3 8.50 3 + +-- !query3_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query3_0_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query3_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query3_1after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query4_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query4_0after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query5_0_before -- + +-- !query5_0_after -- + +-- !query5_1_before -- + +-- !query5_1_after -- + +-- !query6_0_before -- +3 1 1 0.00 +3 1 1 6.50 +3 1 1 6.50 +3 1 4 6.50 +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query6_0_after -- +3 1 1 0.00 +3 1 1 6.50 +3 1 1 6.50 +3 1 4 6.50 +4 2 1 0.00 +4 2 1 6.50 +4 2 1 6.50 +5 2 1 0.00 +5 2 2 8.50 +5 2 2 8.50 +5 2 2 8.50 + +-- !query6_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query6_1_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_0_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_1after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_2_before -- +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query7_2after -- +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query8_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query8_0after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query8_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query8_1after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query9_0_before -- + +-- !query9_0_after -- + +-- !query9_1_before -- + +-- !query9_1_after -- + +-- !query9_2_before -- + +-- !query9_2_after -- + +-- !query10_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query10_0_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query10_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query10_1_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query11_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query11_0_after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query11_1_before -- + +-- !query11_1after -- + +-- !query12_0_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query12_0after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query12_1_before -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + +-- !query12_1after -- +4 2023-12-11 2 1 0.00 +4 2023-12-11 2 1 6.50 +4 2023-12-11 2 1 6.50 +5 2023-12-12 2 1 0.00 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 +5 2023-12-12 2 2 8.50 + diff --git a/regression-test/data/nereids_rules_p0/mv/window/window_above_scan.out b/regression-test/data/nereids_rules_p0/mv/window/window_above_scan.out new file mode 100644 index 00000000000000..4747d5427414fb --- /dev/null +++ b/regression-test/data/nereids_rules_p0/mv/window/window_above_scan.out @@ -0,0 +1,349 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !query1_0_before -- + +-- !query1_0_after -- + +-- !query1_1_before -- + +-- !query1_1_after -- + +-- !query2_0_before -- +3 1 1 +3 1 1 +3 1 1 +3 1 4 +4 2 1 +4 2 1 +4 2 1 +5 2 1 +5 2 2 +5 2 2 +5 2 2 + +-- !query2_0_after -- +3 1 1 +3 1 1 +3 1 1 +3 1 4 +4 2 1 +4 2 1 +4 2 1 +5 2 1 +5 2 2 +5 2 2 +5 2 2 + +-- !query2_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query2_1_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query3_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query3_0_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query3_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query3_1after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query4_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query4_0after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query5_0_before -- + +-- !query5_0_after -- + +-- !query5_1_before -- + +-- !query5_1_after -- + +-- !query6_0_before -- +3 1 1 +3 1 1 +3 1 1 +3 1 4 +4 2 1 +4 2 1 +4 2 1 +5 2 1 +5 2 2 +5 2 2 +5 2 2 + +-- !query6_0_after -- +3 1 1 +3 1 1 +3 1 1 +3 1 4 +4 2 1 +4 2 1 +4 2 1 +5 2 1 +5 2 2 +5 2 2 +5 2 2 + +-- !query6_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query6_1_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_0_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_1after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_2_before -- +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query7_2after -- +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query8_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query8_0after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query8_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query8_1after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query9_0_before -- + +-- !query9_0_after -- + +-- !query9_1_before -- + +-- !query9_1_after -- + +-- !query9_2_before -- + +-- !query9_2_after -- + +-- !query10_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query10_0_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query10_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query10_1_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query11_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query11_0_after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query11_1_before -- + +-- !query11_1after -- + +-- !query12_0_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query12_0after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query12_1_before -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + +-- !query12_1after -- +4 2023-12-11 2 1 +4 2023-12-11 2 1 +4 2023-12-11 2 1 +5 2023-12-12 2 1 +5 2023-12-12 2 2 +5 2023-12-12 2 2 +5 2023-12-12 2 2 + diff --git a/regression-test/suites/nereids_rules_p0/mv/join/null_aware_anti/null_aware_left_anti_join.groovy b/regression-test/suites/nereids_rules_p0/mv/join/null_aware_anti/null_aware_left_anti_join.groovy index 108fc77c863d7a..a019ccb365b8e2 100644 --- a/regression-test/suites/nereids_rules_p0/mv/join/null_aware_anti/null_aware_left_anti_join.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/join/null_aware_anti/null_aware_left_anti_join.groovy @@ -624,7 +624,7 @@ suite("null_aware_anti") { ); """ order_qt_query7_0_before "${query7_0}" - async_mv_rewrite_fail(db, mv7_0, query7_0, "mv7_0") + async_mv_rewrite_success(db, mv7_0, query7_0, "mv7_0") order_qt_query7_0_after "${query7_0}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" @@ -661,7 +661,7 @@ suite("null_aware_anti") { ); """ order_qt_query7_1_before "${query7_1}" - async_mv_rewrite_fail(db, mv7_1, query7_1, "mv7_1") + async_mv_rewrite_success(db, mv7_1, query7_1, "mv7_1") order_qt_query7_1_after "${query7_1}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_1""" diff --git a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/limit/query_with_limit.groovy b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/limit/query_with_limit.groovy index 22f31ae2957d29..f3db2c06aa9f2c 100644 --- a/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/limit/query_with_limit.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/pre_rewrite/limit/query_with_limit.groovy @@ -960,7 +960,7 @@ suite("query_with_limit") { orders on l_orderkey = o_orderkey limit 2 offset 1; """ - async_mv_rewrite_fail(db, mv3_1, query3_1, "mv3_1") + async_mv_rewrite_success(db, mv3_1, query3_1, "mv3_1") sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" @@ -999,7 +999,7 @@ suite("query_with_limit") { order by l_orderkey limit 2 offset 1; """ - async_mv_rewrite_fail(db, mv3_2, query3_2, "mv3_2") + async_mv_rewrite_success(db, mv3_2, query3_2, "mv3_2") sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_2""" // query is union all + limit offset diff --git a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy index eff733a2fec617..ffb2bd5a6d91c8 100644 --- a/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy +++ b/regression-test/suites/nereids_rules_p0/mv/tpch/mv_tpch_test.groovy @@ -228,7 +228,7 @@ suite("mv_tpch_test") { """ // contains limit, doesn't support now order_qt_query2_before "${query2}" - async_mv_rewrite_fail(db, mv2, query2, "mv2") + async_mv_rewrite_success(db, mv2, query2, "mv2") order_qt_query2_after "${query2}" sql """ DROP MATERIALIZED VIEW IF EXISTS mv2""" diff --git a/regression-test/suites/nereids_rules_p0/mv/window/window_above_aggregate.groovy b/regression-test/suites/nereids_rules_p0/mv/window/window_above_aggregate.groovy new file mode 100644 index 00000000000000..033a54c64c242a --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/window/window_above_aggregate.groovy @@ -0,0 +1,1821 @@ +package mv.window +// 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("window_above_aggregate") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set disable_nereids_rules='ELIMINATE_CONST_JOIN_CONDITION,CONSTANT_PROPAGATION'" + + sql """ + drop table if exists orders3 + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders3 ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem3 + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem3 ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp3 + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp3 ( + 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 lineitem3 values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders3 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'), + (1, 1, 'o', 10.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'), + (2, 1, 'o', 11.5, '2023-12-09', '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', 12.5, '2023-12-10', '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'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (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', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp3 values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp3 with sync""" + sql """analyze table lineitem3 with sync""" + sql """analyze table orders3 with sync""" + sql """alter table lineitem3 modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders3 modify column O_COMMENT set stats ('row_count'='18');""" + sql """alter table partsupp3 modify column ps_comment set stats ('row_count'='2');""" + + // multi table or single table + // top filter(project) + window + middle filter(project) + aggregate + bottom filter(project) + scan + // query has only top filter, view has both top and middle filter + def mv1_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 3; + """ + def query1_0 = + """ + select * from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 3; + """ + order_qt_query1_0_before "${query1_0}" + async_mv_rewrite_fail(db, mv1_0, query1_0, "agg_mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv1_0""" + + + def mv1_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 7 + """ + def query1_1 = + """ + select + * + from + ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6; + """ + order_qt_query1_1_before "${query1_1}" + async_mv_rewrite_fail(db, mv1_1, query1_1, "agg_mv1_1") + order_qt_query1_1_after "${query1_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv1_1""" + + + // query has only top filter, view has only top filter + def mv2_0 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query2_0 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 2; + """ + order_qt_query2_0_before "${query2_0}" + async_mv_rewrite_success(db, mv2_0, query2_0, "agg_mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv2_0""" + + + def mv2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query2_1_before "${query2_1}" + async_mv_rewrite_fail(db, mv2_1, query2_1, "agg_mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv2_1""" + + + def mv2_2 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp3 + where ps_suppkey > 1 + group by ps_partkey, ps_suppkey, ps_supplycost + ) window_partsupp3 ON l_partkey = ps_partkey + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, max_suppkey + ) t + where o_orderkey > 1; + """ + def query2_2 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) as max_suppkey + from + partsupp3 + where ps_suppkey > 1 + group by ps_partkey, ps_suppkey, ps_supplycost + ) window_partsupp3 ON l_partkey = ps_partkey + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, max_suppkey + ) t + where o_orderkey > 2; + """ + order_qt_query2_2_before "${query2_2}" + async_mv_rewrite_success(db, mv2_2, query2_2, "agg_mv2_2") + order_qt_query2_2_after "${query2_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv2_2""" + + + def mv2_3 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp3 + where ps_suppkey > 1 + group by ps_partkey, ps_suppkey, ps_supplycost + ) window_partsupp3 ON l_partkey = ps_partkey + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, max_suppkey + ) t + where o_orderkey > 1; + """ + def query2_3 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select * + from + ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp3 + group by ps_partkey, ps_suppkey, ps_supplycost + ) ps_sub + where ps_suppkey > 1 + ) window_partsupp3 ON l_partkey = ps_partkey + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, max_suppkey + ) t + where o_orderkey > 1; + """ + order_qt_query2_3_before "${query2_3}" + // the filter ps_suppkey > 1 on the partsupp3 can not be pushed down, so fail + async_mv_rewrite_fail(db, mv2_3, query2_3, "agg_mv2_3") + order_qt_query2_3_after "${query2_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv2_3""" + + + // query has only top filter, view has only middle filter + def mv3_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query3_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query3_0_before "${query3_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv3_0, query3_0, "agg_mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv3_0""" + + + def mv3_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query3_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 3; + """ + order_qt_query3_1_before "${query3_1}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv3_1, query3_1, "agg_mv3_1") + order_qt_query3_1after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv3_1""" + + + // query has only top filter, view has no filter + def mv4_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query4_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 3; + """ + order_qt_query4_0_before "${query4_0}" + async_mv_rewrite_success(db, mv4_0, query4_0, "agg_mv4_0") + order_qt_query4_0after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv4_0""" + + + def mv4_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query4_1 = + """ + select + MAX(l_extendedprice) as max_extendedprice, + o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, + first_value, rank_value, lag_value + from + ( + select + o_orderkey, + o_orderdate, + o_custkey, + o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey, + first_value, rank_value, lag_value; + """ + order_qt_query4_1_before "${query4_1}" + // because aggregate and window order is different + async_mv_rewrite_fail(db, mv4_1, query4_1, "agg_mv4_1") + order_qt_query4_1after "${query4_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv4_1""" + + + // query has only middle filter, view has both top and middle filter + def mv5_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + def query5_0 = + """ + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query5_0_before "${query5_0}" + async_mv_rewrite_fail(db, mv5_0, query5_0, "agg_mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv5_0""" + + + def mv5_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 7 + """ + def query5_1 = + """ + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query5_1_before "${query5_1}" + async_mv_rewrite_fail(db, mv5_1, query5_1, "agg_mv5_1") + order_qt_query5_1_after "${query5_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv5_1""" + + // query has only middle filter, view has only top filter + def mv6_0 = + """ + select * + from ( + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query6_0 = + """ + select + o_orderkey, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 2 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query6_0_before "${query6_0}" + async_mv_rewrite_fail(db, mv6_0, query6_0, "agg_mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv6_0""" + + + def mv6_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query6_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query6_1_before "${query6_1}" + async_mv_rewrite_fail(db, mv6_1, query6_1, "agg_mv6_1") + order_qt_query6_1_after "${query6_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv6_1""" + + // query has only middle filter, view has only middle filter + def mv7_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query7_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query7_0_before "${query7_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv7_0, query7_0, "agg_mv7_0") + order_qt_query7_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv7_0""" + + def mv7_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query7_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query7_1_before "${query7_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv7_1, query7_1, "agg_mv7_1") + order_qt_query7_1after "${query7_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv7_1""" + + + def mv7_2 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query7_2 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 4 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query7_2_before "${query7_2}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv7_2, query7_2, "agg_mv7_2") + order_qt_query7_2after "${query7_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv7_2""" + + // query has only middle filter, view has no filter + def mv8_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query8_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query8_0_before "${query8_0}" + async_mv_rewrite_fail(db, mv8_0, query8_0, "agg_mv8_0") + order_qt_query8_0after "${query8_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv8_0""" + + + def mv8_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query8_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + order_qt_query8_1_before "${query8_1}" + async_mv_rewrite_success(db, mv8_1, query8_1, "agg_mv8_1") + order_qt_query8_1after "${query8_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv8_1""" + + + // query has both top and middle filter, view has both top and middle filter + def mv9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + def query9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + order_qt_query9_0_before "${query9_0}" + // cbo should chose mv9_0 but not + async_mv_rewrite_success_without_check_chosen(db, mv9_0, query9_0, "agg_mv9_0") + order_qt_query9_0_after "${query9_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv9_0""" + + + def mv9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + def query9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 7 + """ + order_qt_query9_1_before "${query9_1}" + // cbo should chose mv9_0 but not + async_mv_rewrite_success_without_check_chosen(db, mv9_1, query9_1, "agg_mv9_1") + order_qt_query9_1_after "${query9_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv9_1""" + + + def mv9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + def query9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 7 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 6 + """ + order_qt_query9_2_before "${query9_2}" + async_mv_rewrite_fail(db, mv9_2, query9_2, "agg_mv9_2") + order_qt_query9_2_after "${query9_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv9_2""" + + // query has both top and middle filter, view has only top filter + def mv10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 2; + """ + order_qt_query10_0_before "${query10_0}" + async_mv_rewrite_success(db, mv10_0, query10_0, "agg_mv10_0") + order_qt_query10_0_after "${query10_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv10_0""" + + + def mv10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 1; + """ + def query10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 1 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query10_1_before "${query10_1}" + async_mv_rewrite_fail(db, mv10_1, query10_1, "agg_mv10_1") + order_qt_query10_1_after "${query10_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv10_1""" + + + // query has both top and middle filter, view has only middle filter + def mv11_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query11_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 1 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query11_0_before "${query11_0}" + // filter can be pushed down, should success + async_mv_rewrite_fail(db, mv11_0, query11_0, "agg_mv11_0") + order_qt_query11_0_after "${query11_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv11_0""" + + + def mv11_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query11_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 7; + """ + order_qt_query11_1_before "${query11_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv11_1, query11_1, "agg_mv11_1") + order_qt_query11_1after "${query11_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv11_1""" + + // query has both top and middle filter, view has no filter + def mv12_0 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query12_0 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderkey > 3; + """ + order_qt_query12_0_before "${query12_0}" + async_mv_rewrite_success(db, mv12_0, query12_0, "agg_mv12_0") + order_qt_query12_0after "${query12_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv12_0""" + + + def mv12_1 = + """ + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey; + """ + def query12_1 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + o_custkey, + MAX(l_extendedprice) as max_extendedprice, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem3 + left join orders3 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + group by o_orderdate, o_custkey, o_totalprice, o_orderstatus, l_extendedprice, l_shipdate, l_quantity, o_orderkey + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query12_1_before "${query12_1}" + async_mv_rewrite_fail(db, mv12_1, query12_1, "agg_mv12_1") + order_qt_query12_1after "${query12_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS agg_mv12_1""" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/window/window_above_join.groovy b/regression-test/suites/nereids_rules_p0/mv/window/window_above_join.groovy new file mode 100644 index 00000000000000..98326a43b030b5 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/window/window_above_join.groovy @@ -0,0 +1,1591 @@ +package mv.window +// 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("window_above_join") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set disable_nereids_rules='ELIMINATE_CONST_JOIN_CONDITION,CONSTANT_PROPAGATION'" + + sql """ + drop table if exists orders2 + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders2 ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem2 + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem2 ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp2 + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp2 ( + 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 lineitem2 values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders2 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'), + (1, 1, 'o', 10.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'), + (2, 1, 'o', 11.5, '2023-12-09', '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', 12.5, '2023-12-10', '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'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (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', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp2 values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp2 with sync""" + sql """analyze table lineitem2 with sync""" + sql """analyze table orders2 with sync""" + sql """alter table lineitem2 modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders2 modify column O_COMMENT set stats ('row_count'='18');""" + sql """alter table partsupp2 modify column ps_comment set stats ('row_count'='2');""" + + + // top filter(project) + window + bottom filter(project) + join + // query has only top filter, view has both top and bottom filter + def mv1_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 3; + """ + def query1_0 = + """ + select * from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 3; + """ + order_qt_query1_0_before "${query1_0}" + async_mv_rewrite_fail(db, mv1_0, query1_0, "join_mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv1_0""" + + + def mv1_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + ) t + where o_orderkey > 7 + """ + def query1_1 = + """ + select + * + from + ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 6; + """ + order_qt_query1_1_before "${query1_1}" + async_mv_rewrite_fail(db, mv1_1, query1_1, "join_mv1_1") + order_qt_query1_1_after "${query1_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv1_1""" + + + // query has only top filter, view has only top filter + def mv2_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query2_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 2; + """ + order_qt_query2_0_before "${query2_0}" + async_mv_rewrite_success(db, mv2_0, query2_0, "join_mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv2_0""" + + + def mv2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query2_1_before "${query2_1}" + async_mv_rewrite_fail(db, mv2_1, query2_1, "join_mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv2_1""" + + + def mv2_2 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp2 + where ps_suppkey > 1 + ) window_partsupp2 ON l_partkey = ps_partkey + ) t + where o_orderkey > 1; + """ + def query2_2 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp2 + where ps_suppkey > 1 + ) window_partsupp2 ON l_partkey = ps_partkey + ) t + where o_orderkey > 2; + """ + order_qt_query2_2_before "${query2_2}" + async_mv_rewrite_success(db, mv2_2, query2_2, "join_mv2_2") + order_qt_query2_2_after "${query2_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv2_2""" + + + def mv2_3 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp2 + where ps_suppkey > 1 + ) window_partsupp2 ON l_partkey = ps_partkey + ) t + where o_orderkey > 1; + """ + def query2_3 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value, + max_suppkey + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + left join ( + select * + from + ( + select ps_partkey, + ps_suppkey, + MAX(ps_suppkey) OVER ( + PARTITION BY ps_partkey + ORDER BY ps_supplycost NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS max_suppkey + from + partsupp2 + ) ps_sub + where ps_suppkey > 1 + ) window_partsupp2 ON l_partkey = ps_partkey + ) t + where o_orderkey > 1; + """ + order_qt_query2_3_before "${query2_3}" + // the filter ps_suppkey > 1 on the partsupp2 can not be pushed down, so fail + async_mv_rewrite_fail(db, mv2_3, query2_3, "join_mv2_3") + order_qt_query2_3_after "${query2_3}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv2_3""" + + + // query has only top filter, view has only bottom filter + def mv3_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09'; + """ + def query3_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query3_0_before "${query3_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv3_0, query3_0, "join_mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv3_0""" + + + def mv3_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + def query3_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 3; + """ + order_qt_query3_1_before "${query3_1}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv3_1, query3_1, "join_mv3_1") + order_qt_query3_1after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv3_1""" + + + // query has only top filter, view has no filter + def mv4_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query4_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 3; + """ + order_qt_query4_0_before "${query4_0}" + async_mv_rewrite_success(db, mv4_0, query4_0, "join_mv4_0") + order_qt_query4_0after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv4_0""" + + + // query has only bottom filter, view has both top and bottom filter + def mv5_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query5_0 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6; + """ + order_qt_query5_0_before "${query5_0}" + async_mv_rewrite_fail(db, mv5_0, query5_0, "join_mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv5_0""" + + + def mv5_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + ) t + where o_orderkey > 7 + """ + def query5_1 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6; + """ + order_qt_query5_1_before "${query5_1}" + async_mv_rewrite_fail(db, mv5_1, query5_1, "join_mv5_1") + order_qt_query5_1_after "${query5_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv5_1""" + + // query has only bottom filter, view has only top filter + def mv6_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query6_0 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 2; + """ + order_qt_query6_0_before "${query6_0}" + async_mv_rewrite_fail(db, mv6_0, query6_0, "join_mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv6_0""" + + + def mv6_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query6_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10'; + """ + order_qt_query6_1_before "${query6_1}" + async_mv_rewrite_fail(db, mv6_1, query6_1, "join_mv6_1") + order_qt_query6_1_after "${query6_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv6_1""" + + // query has only bottom filter, view has only bottom filter + def mv7_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09'; + """ + def query7_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10'; + """ + order_qt_query7_0_before "${query7_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv7_0, query7_0, "join_mv7_0") + order_qt_query7_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv7_0""" + + def mv7_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + def query7_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + order_qt_query7_1_before "${query7_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv7_1, query7_1, "join_mv7_1") + order_qt_query7_1after "${query7_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv7_1""" + + + def mv7_2 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + def query7_2 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 4; + """ + order_qt_query7_2_before "${query7_2}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv7_2, query7_2, "join_mv7_2") + order_qt_query7_2after "${query7_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv7_2""" + + // query has only bottom filter, view has no filter + def mv8_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query8_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + order_qt_query8_0_before "${query8_0}" + async_mv_rewrite_fail(db, mv8_0, query8_0, "join_mv8_0") + order_qt_query8_0after "${query8_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv8_0""" + + + def mv8_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query8_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10'; + """ + order_qt_query8_1_before "${query8_1}" + async_mv_rewrite_success(db, mv8_1, query8_1, "join_mv8_1") + order_qt_query8_1after "${query8_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv8_1""" + + + // query has both top and bottom filter, view has both top and bottom filter + def mv9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + order_qt_query9_0_before "${query9_0}" + async_mv_rewrite_success_without_check_chosen(db, mv9_0, query9_0, "join_mv9_0") + order_qt_query9_0_after "${query9_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv9_0""" + + + def mv9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 7 + """ + order_qt_query9_1_before "${query9_1}" + async_mv_rewrite_success_without_check_chosen(db, mv9_1, query9_1, "join_mv9_1") + order_qt_query9_1_after "${query9_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv9_1""" + + + def mv9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 6 + ) t + where o_orderkey > 6 + """ + def query9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 7 + ) t + where o_orderkey > 6 + """ + order_qt_query9_2_before "${query9_2}" + async_mv_rewrite_fail(db, mv9_2, query9_2, "join_mv9_2") + order_qt_query9_2_after "${query9_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv9_2""" + + + // query has both top and bottom filter, view has only top filter + def mv10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 2; + """ + order_qt_query10_0_before "${query10_0}" + async_mv_rewrite_success(db, mv10_0, query10_0, "join_mv10_0") + order_qt_query10_0_after "${query10_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv10_0""" + + + def mv10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + ) t + where o_orderkey > 1; + """ + def query10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query10_1_before "${query10_1}" + async_mv_rewrite_fail(db, mv10_1, query10_1, "join_mv10_1") + order_qt_query10_1_after "${query10_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv10_1""" + + + // query has both top and bottom filter, view has only bottom filter + def mv11_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-09'; + """ + def query11_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query11_0_before "${query11_0}" + // filter can be pushed down, should success + async_mv_rewrite_fail(db, mv11_0, query11_0, "join_mv11_0") + order_qt_query11_0_after "${query11_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv11_0""" + + + def mv11_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3; + """ + def query11_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + ) t + where o_orderkey > 7; + """ + order_qt_query11_1_before "${query11_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv11_1, query11_1, "join_mv11_1") + order_qt_query11_1after "${query11_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv11_1""" + + // query has both top and bottom filter, view has no filter + def mv12_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query12_0 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 3; + """ + order_qt_query12_0_before "${query12_0}" + async_mv_rewrite_success(db, mv12_0, query12_0, "join_mv12_0") + order_qt_query12_0after "${query12_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv12_0""" + + + def mv12_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate; + """ + def query12_1 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value, + LAG(l_extendedprice, 1, 0) over (partition by o_orderdate, l_shipdate order by l_quantity) AS lag_value + from + lineitem2 + left join orders2 on l_orderkey = o_orderkey and l_shipdate = o_orderdate + where o_orderkey > 3 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query12_1_before "${query12_1}" + async_mv_rewrite_fail(db, mv12_1, query12_1, "join_mv12_1") + order_qt_query12_1after "${query12_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS join_mv12_1""" +} diff --git a/regression-test/suites/nereids_rules_p0/mv/window/window_above_scan.groovy b/regression-test/suites/nereids_rules_p0/mv/window/window_above_scan.groovy new file mode 100644 index 00000000000000..de1c9225a3e304 --- /dev/null +++ b/regression-test/suites/nereids_rules_p0/mv/window/window_above_scan.groovy @@ -0,0 +1,1329 @@ +package mv.window +// 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("window_above_scan") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + sql "set runtime_filter_mode=OFF"; + sql "SET ignore_shape_nodes='PhysicalDistribute,PhysicalProject'" + sql "set disable_nereids_rules='ELIMINATE_CONST_JOIN_CONDITION,CONSTANT_PROPAGATION'" + + sql """ + drop table if exists orders1 + """ + + sql """ + CREATE TABLE IF NOT EXISTS orders1 ( + o_orderkey INTEGER NOT NULL, + o_custkey INTEGER NOT NULL, + o_orderstatus CHAR(1) NOT NULL, + o_totalprice DECIMALV3(15,2) NOT NULL, + o_orderdate DATE NOT NULL, + o_orderpriority CHAR(15) NOT NULL, + o_clerk CHAR(15) NOT NULL, + o_shippriority INTEGER NOT NULL, + O_COMMENT VARCHAR(79) NOT NULL + ) + DUPLICATE KEY(o_orderkey, o_custkey) + DISTRIBUTED BY HASH(o_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ); + """ + + sql """ + drop table if exists lineitem1 + """ + + sql""" + CREATE TABLE IF NOT EXISTS lineitem1 ( + l_orderkey INTEGER NOT NULL, + l_partkey INTEGER NOT NULL, + l_suppkey INTEGER NOT NULL, + l_linenumber INTEGER NOT NULL, + l_quantity DECIMALV3(15,2) NOT NULL, + l_extendedprice DECIMALV3(15,2) NOT NULL, + l_discount DECIMALV3(15,2) NOT NULL, + l_tax DECIMALV3(15,2) NOT NULL, + l_returnflag CHAR(1) NOT NULL, + l_linestatus CHAR(1) NOT NULL, + l_shipdate DATE NOT NULL, + l_commitdate DATE NOT NULL, + l_receiptdate DATE NOT NULL, + l_shipinstruct CHAR(25) NOT NULL, + l_shipmode CHAR(10) NOT NULL, + l_comment VARCHAR(44) NOT NULL + ) + DUPLICATE KEY(l_orderkey, l_partkey, l_suppkey, l_linenumber) + DISTRIBUTED BY HASH(l_orderkey) BUCKETS 3 + PROPERTIES ( + "replication_num" = "1" + ) + """ + + sql """ + drop table if exists partsupp1 + """ + + sql """ + CREATE TABLE IF NOT EXISTS partsupp1 ( + 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 lineitem1 values + (1, 2, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-08', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (2, 4, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-09', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (3, 2, 4, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-10', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (4, 3, 3, 4, 5.5, 6.5, 7.5, 8.5, 'o', 'k', '2023-12-11', '2023-12-09', '2023-12-10', 'a', 'b', 'yyyyyyyyy'), + (5, 2, 3, 6, 7.5, 8.5, 9.5, 10.5, 'k', 'o', '2023-12-12', '2023-12-12', '2023-12-13', 'c', 'd', 'xxxxxxxxx'); + """ + + sql """ + insert into orders1 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'), + (1, 1, 'o', 10.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'), + (2, 1, 'o', 11.5, '2023-12-09', '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', 12.5, '2023-12-10', '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'), + (4, 2, 'o', 43.2, '2023-12-11', 'c','d',2, 'mm'), + (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', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 56.2, '2023-12-12', 'c','d',2, 'mi'), + (5, 2, 'o', 1.2, '2023-12-12', 'c','d',2, 'mi'); + """ + + sql """ + insert into partsupp1 values + (2, 3, 9, 10.01, 'supply1'), + (2, 3, 10, 11.01, 'supply2'); + """ + + sql """analyze table partsupp1 with sync""" + sql """analyze table lineitem1 with sync""" + sql """analyze table orders1 with sync""" + sql """alter table lineitem1 modify column l_comment set stats ('row_count'='5');""" + sql """alter table orders1 modify column O_COMMENT set stats ('row_count'='18');""" + sql """alter table partsupp1 modify column ps_comment set stats ('row_count'='2');""" + + // top filter(project) + window + bottom filter(project) + scan + // query has only top filter, view has both top and bottom filter + def mv1_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query1_0 = + """ + select * from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 6; + """ + order_qt_query1_0_before "${query1_0}" + async_mv_rewrite_fail(db, mv1_0, query1_0, "mv1_0") + order_qt_query1_0_after "${query1_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_0""" + + + def mv1_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 6 + ) t + where o_orderkey > 7 + """ + def query1_1 = + """ + select + * + from + ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 6; + """ + order_qt_query1_1_before "${query1_1}" + async_mv_rewrite_fail(db, mv1_1, query1_1, "mv1_1") + order_qt_query1_1_after "${query1_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv1_1""" + + + // query has only top filter, view has only top filter + def mv2_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query2_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 2; + """ + order_qt_query2_0_before "${query2_0}" + async_mv_rewrite_success(db, mv2_0, query2_0, "mv2_0") + order_qt_query2_0_after "${query2_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_0""" + + + def mv2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query2_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query2_1_before "${query2_1}" + async_mv_rewrite_fail(db, mv2_1, query2_1, "mv2_1") + order_qt_query2_1_after "${query2_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv2_1""" + + // query has only top filter, view has only bottom filter + def mv3_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-09'; + """ + def query3_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query3_0_before "${query3_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv3_0, query3_0, "mv3_0") + order_qt_query3_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_0""" + + + def mv3_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + def query3_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 3; + """ + order_qt_query3_1_before "${query3_1}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv3_1, query3_1, "mv3_1") + order_qt_query3_1after "${query3_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv3_1""" + + + // query has only top filter, view has no filter + def mv4_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1; + """ + def query4_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 3; + """ + order_qt_query4_0_before "${query4_0}" + async_mv_rewrite_success(db, mv4_0, query4_0, "mv4_0") + order_qt_query4_0after "${query4_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv4_0""" + + + // query has only bottom filter, view has both top and bottom filter + def mv5_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query5_0 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 6; + """ + order_qt_query5_0_before "${query5_0}" + async_mv_rewrite_fail(db, mv5_0, query5_0, "mv5_0") + order_qt_query5_0_after "${query5_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_0""" + + + def mv5_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 6 + ) t + where o_orderkey > 7 + """ + def query5_1 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 6; + """ + order_qt_query5_1_before "${query5_1}" + async_mv_rewrite_fail(db, mv5_1, query5_1, "mv5_1") + order_qt_query5_1_after "${query5_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv5_1""" + + // query has only bottom filter, view has only top filter + def mv6_0 = + """ + select * + from ( + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query6_0 = + """ + select + o_orderkey, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 2; + """ + order_qt_query6_0_before "${query6_0}" + async_mv_rewrite_fail(db, mv6_0, query6_0, "mv6_0") + order_qt_query6_0_after "${query6_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_0""" + + + def mv6_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query6_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10'; + """ + order_qt_query6_1_before "${query6_1}" + async_mv_rewrite_fail(db, mv6_1, query6_1, "mv6_1") + order_qt_query6_1_after "${query6_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv6_1""" + + // query has only bottom filter, view has only bottom filter + def mv7_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-09'; + """ + def query7_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10'; + """ + order_qt_query7_0_before "${query7_0}" + // filter can be pushed down, should success + async_mv_rewrite_success(db, mv7_0, query7_0, "mv7_0") + order_qt_query7_0_after "${query3_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_0""" + + def mv7_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + def query7_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + order_qt_query7_1_before "${query7_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv7_1, query7_1, "mv7_1") + order_qt_query7_1after "${query7_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_1""" + + + def mv7_2 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + def query7_2 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 4; + """ + order_qt_query7_2_before "${query7_2}" + // filter can not be pushed down, should success + async_mv_rewrite_fail(db, mv7_2, query7_2, "mv7_2") + order_qt_query7_2after "${query7_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv7_2""" + + // query has only bottom filter, view has no filter + def mv8_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1; + """ + def query8_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + order_qt_query8_0_before "${query8_0}" + async_mv_rewrite_fail(db, mv8_0, query8_0, "mv8_0") + order_qt_query8_0after "${query8_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv8_0""" + + + def mv8_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1; + """ + def query8_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10'; + """ + order_qt_query8_1_before "${query8_1}" + async_mv_rewrite_success(db, mv8_1, query8_1, "mv8_1") + order_qt_query8_1after "${query8_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv8_1""" + + + // query has both top and bottom filter, view has both top and bottom filter + def mv9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query9_0 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + order_qt_query9_0_before "${query9_0}" + async_mv_rewrite_success_without_check_chosen(db, mv9_0, query9_0, "mv9_0") + order_qt_query9_0_after "${query9_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv9_0""" + + + def mv9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 6 + """ + def query9_1 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 7 + """ + order_qt_query9_1_before "${query9_1}" + async_mv_rewrite_success_without_check_chosen(db, mv9_1, query9_1, "mv9_1") + order_qt_query9_1_after "${query9_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv9_1""" + + + def mv9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 6 + ) t + where o_orderkey > 6 + """ + def query9_2 = + """ + select * from ( + select + o_orderkey, o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 7 + ) t + where o_orderkey > 6 + """ + order_qt_query9_2_before "${query9_2}" + async_mv_rewrite_fail(db, mv9_2, query9_2, "mv9_2") + order_qt_query9_2_after "${query9_2}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv9_2""" + + + // query has both top and bottom filter, view has only top filter + def mv10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query10_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 2; + """ + order_qt_query10_0_before "${query10_0}" + async_mv_rewrite_success(db, mv10_0, query10_0, "mv10_0") + order_qt_query10_0_after "${query10_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv10_0""" + + + def mv10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + ) t + where o_orderkey > 1; + """ + def query10_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query10_1_before "${query10_1}" + async_mv_rewrite_fail(db, mv10_1, query10_1, "mv10_1") + order_qt_query10_1_after "${query10_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv10_1""" + + + // query has both top and bottom filter, view has only bottom filter + def mv11_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-09'; + """ + def query11_0 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 1 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query11_0_before "${query11_0}" + // filter can be pushed down, should success + async_mv_rewrite_fail(db, mv11_0, query11_0, "mv11_0") + order_qt_query11_0_after "${query11_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv11_0""" + + + def mv11_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3; + """ + def query11_1 = + """ + select * + from ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3 + ) t + where o_orderkey > 7; + """ + order_qt_query11_1_before "${query11_1}" + // filter can not be pushed down, should success + async_mv_rewrite_success(db, mv11_1, query11_1, "mv11_1") + order_qt_query11_1after "${query11_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv11_1""" + + // query has both top and bottom filter, view has no filter + def mv12_0 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1; + """ + def query12_0 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderdate > '2023-12-10' + ) t + where o_orderkey > 3; + """ + order_qt_query12_0_before "${query12_0}" + async_mv_rewrite_success(db, mv12_0, query12_0, "mv12_0") + order_qt_query12_0after "${query12_0}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv12_0""" + + + def mv12_1 = + """ + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1; + """ + def query12_1 = + """ + select + * + from + ( + select + o_orderkey, + o_orderdate, + FIRST_VALUE(o_custkey) OVER ( + PARTITION BY o_orderdate + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS first_value, + RANK() OVER ( + PARTITION BY o_orderdate, o_orderstatus + ORDER BY o_totalprice NULLS LAST + RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW + ) AS rank_value + from + orders1 + where o_orderkey > 3 + ) t + where o_orderdate > '2023-12-10'; + """ + order_qt_query12_1_before "${query12_1}" + async_mv_rewrite_fail(db, mv12_1, query12_1, "mv12_1") + order_qt_query12_1after "${query12_1}" + sql """ DROP MATERIALIZED VIEW IF EXISTS mv12_1""" + +}