Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -44,11 +44,15 @@
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterProjectAggregateRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterProjectJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterProjectScanRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewFilterScanRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewOnlyJoinRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectAggregateRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterAggregateRule;
import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectFilterJoinRule;
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.expression.ExpressionOptimization;
import org.apache.doris.nereids.rules.implementation.AggregateStrategies;
import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows;
Expand Down Expand Up @@ -243,6 +247,10 @@ public class RuleSet {
.add(MaterializedViewFilterAggregateRule.INSTANCE)
.add(MaterializedViewProjectFilterAggregateRule.INSTANCE)
.add(MaterializedViewFilterProjectAggregateRule.INSTANCE)
.add(MaterializedViewFilterScanRule.INSTANCE)
.add(MaterializedViewFilterProjectScanRule.INSTANCE)
.add(MaterializedViewProjectScanRule.INSTANCE)
.add(MaterializedViewProjectFilterScanRule.INSTANCE)
.build();

public static final List<Rule> DPHYP_REORDER_RULES = ImmutableList.<Rule>builder()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.nereids.rules.exploration.mv;

import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;

import com.google.common.collect.ImmutableList;

import java.util.List;

/**
* MaterializedViewFilterProjectScanRule
*/
public class MaterializedViewFilterProjectScanRule extends MaterializedViewScanRule {

public static final MaterializedViewFilterProjectScanRule INSTANCE = new MaterializedViewFilterProjectScanRule();

@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalFilter(logicalProject(logicalOlapScan())).thenApplyMultiNoThrow(ctx -> {
LogicalFilter<LogicalProject<LogicalOlapScan>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_FILTER_PROJECT_SCAN));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.nereids.rules.exploration.mv;

import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;

import com.google.common.collect.ImmutableList;

import java.util.List;

/**
* MaterializedViewFilterScanRule
*/
public class MaterializedViewFilterScanRule extends MaterializedViewScanRule {

public static final MaterializedViewFilterScanRule INSTANCE = new MaterializedViewFilterScanRule();

@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalFilter(logicalOlapScan()).thenApplyMultiNoThrow(ctx -> {
LogicalFilter<LogicalOlapScan> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_FILTER_SCAN));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.nereids.rules.exploration.mv;

import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;

import com.google.common.collect.ImmutableList;

import java.util.List;

/**
* MaterializedViewProjectFilterScanRule
*/
public class MaterializedViewProjectFilterScanRule extends MaterializedViewScanRule {

public static final MaterializedViewProjectFilterScanRule INSTANCE = new MaterializedViewProjectFilterScanRule();

@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalProject(logicalFilter(logicalOlapScan())).thenApplyMultiNoThrow(ctx -> {
LogicalProject<LogicalFilter<LogicalOlapScan>> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_FILTER_SCAN));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,44 @@
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.

package org.apache.doris.nereids.rules.exploration.mv;

import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;

import com.google.common.collect.ImmutableList;

import java.util.List;

/**
* MaterializedViewProjectScanRule
*/
public class MaterializedViewProjectScanRule extends MaterializedViewScanRule {

public static final MaterializedViewProjectScanRule INSTANCE = new MaterializedViewProjectScanRule();

@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalProject(logicalOlapScan()).thenApplyMultiNoThrow(ctx -> {
LogicalProject<LogicalOlapScan> root = ctx.root;
return rewrite(root, ctx.cascadesContext);
}).toRule(RuleType.MATERIALIZED_VIEW_PROJECT_SCAN));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,67 @@

package org.apache.doris.nereids.rules.exploration.mv;

import org.apache.doris.nereids.rules.Rule;
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.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;

import com.google.common.collect.ImmutableSet;

import java.util.List;
import java.util.stream.Collectors;

/**
* This is responsible for single table rewriting according to different pattern
* */
public class MaterializedViewScanRule extends AbstractMaterializedViewRule {
public abstract class MaterializedViewScanRule extends AbstractMaterializedViewRule {

@Override
protected Plan rewriteQueryByView(MatchMode matchMode,
StructInfo queryStructInfo,
StructInfo viewStructInfo,
SlotMapping targetToSourceMapping,
Plan tempRewritedPlan,
MaterializationContext materializationContext) {
// Rewrite top projects, represent the query projects by view
List<Expression> expressionsRewritten = rewriteExpression(
queryStructInfo.getExpressions(),
queryStructInfo.getTopPlan(),
materializationContext.getMvExprToMvScanExprMapping(),
targetToSourceMapping,
true,
queryStructInfo.getTableBitSet()
);
// Can not rewrite, bail out
if (expressionsRewritten.isEmpty()) {
materializationContext.recordFailReason(queryStructInfo,
"Rewrite expressions by view in scan fail",
() -> String.format("expressionToRewritten is %s,\n mvExprToMvScanExprMapping is %s,\n"
+ "targetToSourceMapping = %s", queryStructInfo.getExpressions(),
materializationContext.getMvExprToMvScanExprMapping(),
targetToSourceMapping));
return null;
}
return new LogicalProject<>(
expressionsRewritten.stream()
.map(expression -> expression instanceof NamedExpression ? expression : new Alias(expression))
.map(NamedExpression.class::cast)
.collect(Collectors.toList()),
tempRewritedPlan);
}

/**
* Check scan is whether valid or not. Support join's input only support project, filter, join,
* logical relation, simple aggregate node. Con not have aggregate above on join.
* Join condition should be slot reference equals currently.
*/
@Override
public List<Rule> buildRules() {
return null;
protected boolean checkPattern(StructInfo structInfo) {
PlanCheckContext checkContext = PlanCheckContext.of(ImmutableSet.of());
return structInfo.getTopPlan().accept(StructInfo.SCAN_PLAN_PATTERN_CHECKER, checkContext)
&& !checkContext.isContainsTopAggregate();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,7 @@
*/
public class StructInfo {
public static final PlanPatternChecker PLAN_PATTERN_CHECKER = new PlanPatternChecker();
public static final ScanPlanPatternChecker SCAN_PLAN_PATTERN_CHECKER = new ScanPlanPatternChecker();
// struct info splitter
public static final PlanSplitter PLAN_SPLITTER = new PlanSplitter();
private static final RelationCollector RELATION_COLLECTOR = new RelationCollector();
Expand Down Expand Up @@ -606,6 +607,39 @@ private Boolean doVisit(Plan plan, PlanCheckContext checkContext) {
}
}

/**
* ScanPlanPatternChecker, this is used to check the plan pattern is valid or not
*/
public static class ScanPlanPatternChecker extends DefaultPlanVisitor<Boolean, PlanCheckContext> {

@Override
public Boolean visitGroupPlan(GroupPlan groupPlan, PlanCheckContext checkContext) {
return groupPlan.getGroup().getLogicalExpressions().stream()
.anyMatch(logicalExpression -> logicalExpression.getPlan().accept(this, checkContext));
}

@Override
public Boolean visit(Plan plan, PlanCheckContext checkContext) {
if (plan instanceof Filter
|| plan instanceof Project
|| plan instanceof CatalogRelation
|| plan instanceof GroupPlan) {
return doVisit(plan, checkContext);
}
return false;
}

private Boolean doVisit(Plan plan, PlanCheckContext checkContext) {
for (Plan child : plan.children()) {
boolean valid = child.accept(this, checkContext);
if (!valid) {
return false;
}
}
return true;
}
}

/**
* Add predicates on base table when materialized view scan contains invalid partitions
*/
Expand Down
33 changes: 33 additions & 0 deletions regression-test/data/nereids_rules_p0/mv/scan/scan_table.out
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
-- This file is automatically generated. You should know what you did if you want to edit this
-- !query1_0_before --
4 3 3 4 5.50 6.50 7.50 8.50 o k 2023-12-11 2023-12-09 2023-12-10 a b yyyyyyyyy
5 2 3 6 7.50 8.50 9.50 10.50 k o 2023-12-12 2023-12-12 2023-12-13 c d xxxxxxxxx

-- !query1_0_after --
4 3 3 4 5.50 6.50 7.50 8.50 o k 2023-12-11 2023-12-09 2023-12-10 a b yyyyyyyyy
5 2 3 6 7.50 8.50 9.50 10.50 k o 2023-12-12 2023-12-12 2023-12-13 c d xxxxxxxxx

-- !query1_1_before --
1 2 3 4 5.50 6.50 7.50 8.50 o k 2023-12-08 2023-12-09 2023-12-10 a b yyyyyyyyy
2 4 3 4 5.50 6.50 7.50 8.50 o k 2023-12-09 2023-12-09 2023-12-10 a b yyyyyyyyy
3 2 4 4 5.50 6.50 7.50 8.50 o k 2023-12-10 2023-12-09 2023-12-10 a b yyyyyyyyy
4 3 3 4 5.50 6.50 7.50 8.50 o k 2023-12-11 2023-12-09 2023-12-10 a b yyyyyyyyy

-- !query1_1_after --
1 2 3 4 5.50 6.50 7.50 8.50 o k 2023-12-08 2023-12-09 2023-12-10 a b yyyyyyyyy
2 4 3 4 5.50 6.50 7.50 8.50 o k 2023-12-09 2023-12-09 2023-12-10 a b yyyyyyyyy
3 2 4 4 5.50 6.50 7.50 8.50 o k 2023-12-10 2023-12-09 2023-12-10 a b yyyyyyyyy
4 3 3 4 5.50 6.50 7.50 8.50 o k 2023-12-11 2023-12-09 2023-12-10 a b yyyyyyyyy

-- !query1_2_before --
6 2023-12-13

-- !query1_2_after --
6 2023-12-13

-- !query1_3_before --
6

-- !query1_3_after --
6

Loading