diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index ebef71feb850b0..34c09ad4f75487 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -44,6 +44,7 @@ import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; +import org.apache.doris.nereids.rules.exploration.mv.MaterializationContext; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -112,6 +113,8 @@ public class CascadesContext implements ScheduleContext { private final Optional currentTree; private final Optional parent; + private List materializationContexts; + /** * Constructor of OptimizerContext. * @@ -133,6 +136,7 @@ private CascadesContext(Optional parent, Optional curren this.currentJobContext = new JobContext(this, requireProperties, Double.MAX_VALUE); this.subqueryExprIsAnalyzed = new HashMap<>(); this.runtimeFilterContext = new RuntimeFilterContext(getConnectContext().getSessionVariable()); + this.materializationContexts = new ArrayList<>(); } /** @@ -309,6 +313,14 @@ public void setOuterScope(@Nullable Scope outerScope) { this.outerScope = Optional.ofNullable(outerScope); } + public List getMaterializationContexts() { + return materializationContexts; + } + + public void addMaterializationContext(MaterializationContext materializationContext) { + this.materializationContexts.add(materializationContext); + } + /** * getAndCacheSessionVariable */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index bf870f9dd6b5c7..70166aade87eee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -259,6 +259,10 @@ private void initCascadesContext(LogicalPlan plan, PhysicalProperties requirePro if (statementContext.getConnectContext().getTables() != null) { cascadesContext.setTables(statementContext.getConnectContext().getTables()); } + if (statementContext.getConnectContext().getSessionVariable().enableMaterializedViewRewrite) { + // TODO Pre handle materialized view to materializationContext and + // call cascadesContext.addMaterializationContext() to add it + } } private void analyze() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java index 38c0c4484bc609..08b7731fd0915d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/OptimizeGroupExpressionJob.java @@ -46,6 +46,9 @@ public void execute() { countJobExecutionTimesOfGroupExpressions(groupExpression); List implementationRules = getRuleSet().getImplementationRules(); List explorationRules = getExplorationRules(); + if (context.getCascadesContext().getConnectContext().getSessionVariable().isEnableMaterializedViewRewrite()) { + explorationRules.addAll(getRuleSet().getMaterializedViewRules()); + } for (Rule rule : explorationRules) { if (rule.isInvalid(disableRules, groupExpression)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java index 8009156e134063..47c7b88b690a5a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.cost.Cost; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.exploration.mv.StructInfo; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; @@ -74,6 +75,8 @@ public class Group { private int chosenGroupExpressionId = -1; + private Optional structInfo = Optional.empty(); + /** * Constructor for Group. * @@ -532,4 +535,12 @@ public String treeString() { return TreeStringUtils.treeString(this, toString, getChildren, getExtraPlans, displayExtraPlan); } + + public Optional getStructInfo() { + return structInfo; + } + + public void setStructInfo(StructInfo structInfo) { + this.structInfo = Optional.ofNullable(structInfo); + } } 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 80163663184227..726c0d65e6a7b5 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 @@ -40,6 +40,7 @@ import org.apache.doris.nereids.rules.exploration.join.PushdownProjectThroughSemiJoin; import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTranspose; import org.apache.doris.nereids.rules.exploration.join.SemiJoinSemiJoinTransposeProject; +import org.apache.doris.nereids.rules.exploration.mv.MaterializedViewProjectJoinRule; import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; import org.apache.doris.nereids.rules.implementation.LogicalCTEAnchorToPhysicalCTEAnchor; @@ -220,6 +221,10 @@ public class RuleSet { .add(JoinCommute.BUSHY.build()) .build(); + public static final List MATERIALIZED_VIEW_RULES = planRuleFactories() + .add(MaterializedViewProjectJoinRule.INSTANCE) + .build(); + public List getDPHypReorderRules() { return DPHYP_REORDER_RULES; } @@ -240,6 +245,10 @@ public List getImplementationRules() { return IMPLEMENTATION_RULES; } + public List getMaterializedViewRules() { + return MATERIALIZED_VIEW_RULES; + } + public static RuleFactories planRuleFactories() { return new RuleFactories(); } 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 94ccef528fd996..edad74e2e19804 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 @@ -225,6 +225,24 @@ public enum RuleType { MATERIALIZED_INDEX_PROJECT_SCAN(RuleTypeClass.REWRITE), MATERIALIZED_INDEX_PROJECT_FILTER_SCAN(RuleTypeClass.REWRITE), MATERIALIZED_INDEX_FILTER_PROJECT_SCAN(RuleTypeClass.REWRITE), + + MATERIALIZED_VIEW_PROJECT_JOIN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_FILTER_JOIN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_PROJECT_FILTER_JOIN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_FILTER_PROJECT_JOIN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_ONLY_JOIN(RuleTypeClass.REWRITE), + + MATERIALIZED_VIEW_PROJECT_AGGREGATE(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_FILTER_AGGREGATE(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_PROJECT_FILTER_AGGREGATE(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_FILTER_PROJECT_AGGREGATE(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_ONLY_AGGREGATE(RuleTypeClass.REWRITE), + + MATERIALIZED_VIEW_FILTER_SCAN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_PROJECT_SCAN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_FILTER_PROJECT_SCAN(RuleTypeClass.REWRITE), + MATERIALIZED_VIEW_PROJECT_FILTER_SCAN(RuleTypeClass.REWRITE), + OLAP_SCAN_PARTITION_PRUNE(RuleTypeClass.REWRITE), FILE_SCAN_PARTITION_PRUNE(RuleTypeClass.REWRITE), PUSH_CONJUNCTS_INTO_JDBC_SCAN(RuleTypeClass.REWRITE), 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 new file mode 100644 index 00000000000000..a9a8b754d33648 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewAggregateRule.java @@ -0,0 +1,25 @@ +// 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; + +/** + * AbstractMaterializedViewAggregateRule + * This is responsible for common aggregate rewriting + * */ +public abstract class AbstractMaterializedViewAggregateRule extends AbstractMaterializedViewRule { +} 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 new file mode 100644 index 00000000000000..3080377e32018a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewJoinRule.java @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +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 java.util.List; + +/** + * AbstractMaterializedViewJoinRule + * This is responsible for common join rewriting + */ +public abstract class AbstractMaterializedViewJoinRule extends AbstractMaterializedViewRule { + + @Override + protected Plan rewriteQueryByView(MatchMode matchMode, + StructInfo queryStructInfo, + StructInfo viewStructInfo, + RelationMapping queryToViewTableMappings, + Plan tempRewritedPlan) { + + // Rewrite top projects, represent the query projects by view + List expressions = rewriteExpression( + queryStructInfo.getExpressions(), + queryStructInfo, + viewStructInfo, + queryToViewTableMappings, + tempRewritedPlan + ); + // Can not rewrite, bail out + if (expressions == null) { + return null; + } + return new LogicalProject<>(expressions, tempRewritedPlan); + } + + // Check join is whether valid or not. Support join's input can not contain aggregate + // Only support project, filter, join, logical relation node and + // join condition should be slot reference equals currently + @Override + protected boolean isPatternSupport(StructInfo structInfo) { + // TODO Should get struct info from hyper graph and check + return false; + } +} 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 new file mode 100644 index 00000000000000..8275df732637b5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/AbstractMaterializedViewRule.java @@ -0,0 +1,238 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.rules.exploration.mv.Mapping.ExpressionIndexMapping; +import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; +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.algebra.CatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** + * The abstract class for all materialized view rules + */ +public abstract class AbstractMaterializedViewRule { + + protected List rewrite(Plan queryPlan, CascadesContext cascadesContext) { + List materializationContexts = cascadesContext.getMaterializationContexts(); + List rewriteResults = new ArrayList<>(); + if (materializationContexts.isEmpty()) { + return rewriteResults; + } + StructInfo queryStructInfo = extractStructInfo(queryPlan, cascadesContext); + // Check query queryPlan + if (!isPatternSupport(queryStructInfo)) { + return rewriteResults; + } + + for (MaterializationContext materializationContext : materializationContexts) { + Plan mvPlan = materializationContext.getMvPlan(); + StructInfo viewStructInfo = extractStructInfo(mvPlan, cascadesContext); + if (!isPatternSupport(viewStructInfo)) { + continue; + } + if (!StructInfo.isGraphLogicalEquals(queryStructInfo.getHyperGraph(), viewStructInfo.getHyperGraph())) { + continue; + } + MatchMode matchMode = decideMatchMode(queryStructInfo.getRelations(), viewStructInfo.getRelations()); + if (MatchMode.NOT_MATCH == matchMode) { + continue; + } + List queryToViewTableMappings = + RelationMapping.generate(queryStructInfo.getRelations(), viewStructInfo.getRelations()); + for (RelationMapping queryToViewTableMapping : queryToViewTableMappings) { + SplitPredicate compensatePredicates = predicatesCompensate(queryStructInfo, viewStructInfo, + queryToViewTableMapping); + // can not compensate, bail out + if (compensatePredicates == null || compensatePredicates.isEmpty()) { + continue; + } + Plan rewritedPlan; + Plan mvScan = materializationContext.getScanPlan(); + if (compensatePredicates.isAlwaysTrue()) { + rewritedPlan = mvScan; + } else { + // try to rewrite compensate predicates by using mv scan + List rewriteCompensatePredicates = rewriteExpression( + compensatePredicates.toList(), + queryStructInfo, + viewStructInfo, + queryToViewTableMapping, + mvScan); + if (rewriteCompensatePredicates.isEmpty()) { + continue; + } + rewritedPlan = new LogicalFilter<>(Sets.newHashSet(rewriteCompensatePredicates), mvScan); + } + // rewrite query by view + rewritedPlan = rewriteQueryByView(matchMode, queryStructInfo, viewStructInfo, + queryToViewTableMapping, rewritedPlan); + if (rewritedPlan == null) { + continue; + } + rewriteResults.add(rewritedPlan); + } + } + return rewriteResults; + } + + // Rewrite query by view, for aggregate or join rewriting should be different inherit class implementation + protected Plan rewriteQueryByView(MatchMode matchMode, + StructInfo queryStructInfo, + StructInfo viewStructInfo, + RelationMapping queryToViewTableMappings, + Plan tempRewritedPlan) { + return tempRewritedPlan; + } + + // Use target targetScanNode output expression to represent the source expression + protected List rewriteExpression(List sourceExpressions, + StructInfo sourceStructInfo, + StructInfo targetStructInfo, + RelationMapping sourceToTargetMapping, + Plan targetScanNode) { + // TODO represent the sourceExpressions by using target scan node + // Firstly, rewrite the target plan output expression using query with inverse mapping + // then try to use the mv expression to represent the query. if any of source expressions + // can not be represented by mv, return null + // + // example as following: + // source target + // project(slot 1, 2) project(slot 3, 2, 1) + // scan(table) scan(table) + // + // transform source to: + // project(slot 2, 1) + // target + List targetTopExpressions = targetStructInfo.getExpressions(); + List shuttledTargetExpressions = ExpressionUtils.shuttleExpressionWithLineage( + targetTopExpressions, targetStructInfo.getOriginalPlan(), Sets.newHashSet(), Sets.newHashSet()); + SlotMapping sourceToTargetSlotMapping = SlotMapping.generate(sourceToTargetMapping); + // mv sql plan expressions transform to query based + List queryBasedExpressions = ExpressionUtils.permute(shuttledTargetExpressions, + sourceToTargetSlotMapping.inverse()); + // mv sql query based expression and index mapping + ExpressionIndexMapping.generate(queryBasedExpressions); + // TODO visit source expression and replace the expression with expressionIndexMapping + return ImmutableList.of(); + } + + /** + * Compensate mv predicates by query predicates, compensate predicate is query based. + * Such as a > 5 in mv, and a > 10 in query, the compensatory predicate is a > 10 + * and a = b in mv, and a = b and c = d in query, the compensatory predicate is c = d + */ + protected SplitPredicate predicatesCompensate( + StructInfo queryStructInfo, + StructInfo viewStructInfo, + RelationMapping queryToViewTableMapping + ) { + // TODO Equal predicate compensate + EquivalenceClass queryEquivalenceClass = queryStructInfo.getEquivalenceClass(); + EquivalenceClass viewEquivalenceClass = viewStructInfo.getEquivalenceClass(); + if (queryEquivalenceClass.isEmpty() + && !viewEquivalenceClass.isEmpty()) { + return null; + } + // TODO range predicates and residual predicates compensate + return SplitPredicate.empty(); + } + + private MatchMode decideMatchMode(List queryRelations, List viewRelations) { + List queryTableRefs = queryRelations + .stream() + .map(CatalogRelation::getTable) + .collect(Collectors.toList()); + List viewTableRefs = viewRelations + .stream() + .map(CatalogRelation::getTable) + .collect(Collectors.toList()); + boolean sizeSame = viewTableRefs.size() == queryTableRefs.size(); + boolean queryPartial = viewTableRefs.containsAll(queryTableRefs); + if (!sizeSame && queryPartial) { + return MatchMode.QUERY_PARTIAL; + } + boolean viewPartial = queryTableRefs.containsAll(viewTableRefs); + if (!sizeSame && viewPartial) { + return MatchMode.VIEW_PARTIAL; + } + if (sizeSame && queryPartial && viewPartial) { + return MatchMode.COMPLETE; + } + return MatchMode.NOT_MATCH; + } + + protected StructInfo extractStructInfo(Plan plan, CascadesContext cascadesContext) { + + if (plan.getGroupExpression().isPresent() + && plan.getGroupExpression().get().getOwnerGroup().getStructInfo().isPresent()) { + Group belongGroup = plan.getGroupExpression().get().getOwnerGroup(); + return belongGroup.getStructInfo().get(); + } else { + // TODO build graph from plan and extract struct from graph and set to group if exist + // Should get structInfo from hyper graph and add into current group + StructInfo structInfo = StructInfo.of(plan); + if (plan.getGroupExpression().isPresent()) { + plan.getGroupExpression().get().getOwnerGroup().setStructInfo(structInfo); + } + return structInfo; + } + } + + protected boolean isPatternSupport(StructInfo structInfo) { + if (structInfo.getRelations().isEmpty()) { + return false; + } + return false; + } + + /** + * Query and mv match node + */ + protected enum MatchMode { + /** + * The tables in query are same to the tables in view + */ + COMPLETE, + /** + * The tables in query contains all the tables in view + */ + VIEW_PARTIAL, + /** + * The tables in view contains all the tables in query + */ + QUERY_PARTIAL, + /** + * Except for COMPLETE and VIEW_PARTIAL and QUERY_PARTIAL + */ + NOT_MATCH + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/EquivalenceClass.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/EquivalenceClass.java new file mode 100644 index 00000000000000..2d9880c23bcaa4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/EquivalenceClass.java @@ -0,0 +1,92 @@ +// 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.trees.expressions.SlotReference; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * EquivalenceClass, this is used for equality propagation when predicate compensation + */ +public class EquivalenceClass { + + private final Map> equivalenceSlotMap = new LinkedHashMap<>(); + + public EquivalenceClass() { + } + + /** + * EquivalenceClass + */ + public void addEquivalenceClass(SlotReference slot0, SlotReference slot1) { + + Set slot0Sets = equivalenceSlotMap.get(slot0); + Set slot1Sets = equivalenceSlotMap.get(slot1); + if (slot0Sets != null && slot1Sets != null) { + // Both present, we need to merge + if (slot0Sets.size() < slot1Sets.size()) { + // We swap them to merge + Set tmp = slot1Sets; + slot1Sets = slot0Sets; + slot0Sets = tmp; + } + for (SlotReference newRef : slot1Sets) { + slot0Sets.add(newRef); + equivalenceSlotMap.put(newRef, slot0Sets); + } + } else if (slot0Sets != null) { + // p1 present, we need to merge into it + slot0Sets.add(slot1); + equivalenceSlotMap.put(slot1, slot0Sets); + } else if (slot1Sets != null) { + // p2 present, we need to merge into it + slot1Sets.add(slot0); + equivalenceSlotMap.put(slot0, slot1Sets); + } else { + // None are present, add to same equivalence class + Set equivalenceClass = new LinkedHashSet<>(); + equivalenceClass.add(slot0); + equivalenceClass.add(slot1); + equivalenceSlotMap.put(slot0, equivalenceClass); + equivalenceSlotMap.put(slot1, equivalenceClass); + } + } + + public Map> getEquivalenceSlotMap() { + return equivalenceSlotMap; + } + + public boolean isEmpty() { + return equivalenceSlotMap.isEmpty(); + } + + /** + * EquivalenceClass + */ + public List> getEquivalenceValues() { + List> values = new ArrayList<>(); + equivalenceSlotMap.values().forEach(each -> values.add(each)); + return values; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Mapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Mapping.java new file mode 100644 index 00000000000000..487fc92ce50a2b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Mapping.java @@ -0,0 +1,144 @@ +// 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.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; + +import java.util.List; +import java.util.Objects; + +/** + * Mapping slot from query to view or inversely, + * it can also represent the mapping from slot to it's index + */ +public abstract class Mapping { + + /** + * The relation for mapping + */ + public static final class MappedRelation { + public final RelationId relationId; + public final CatalogRelation belongedRelation; + + public MappedRelation(RelationId relationId, CatalogRelation belongedRelation) { + this.relationId = relationId; + this.belongedRelation = belongedRelation; + } + + public MappedRelation of(RelationId relationId, CatalogRelation belongedRelation) { + return new MappedRelation(relationId, belongedRelation); + } + + public RelationId getRelationId() { + return relationId; + } + + public CatalogRelation getBelongedRelation() { + return belongedRelation; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MappedRelation that = (MappedRelation) o; + return Objects.equals(relationId, that.relationId); + } + + @Override + public int hashCode() { + return Objects.hash(relationId); + } + } + + /** + * The slot for mapping + */ + public static final class MappedSlot { + + public final ExprId exprId; + public final CatalogRelation belongedRelation; + + public MappedSlot(ExprId exprId, CatalogRelation belongedRelation) { + this.exprId = exprId; + this.belongedRelation = belongedRelation; + } + + public MappedSlot of(ExprId exprId, CatalogRelation belongedRelation) { + return new MappedSlot(exprId, belongedRelation); + } + + public ExprId getExprId() { + return exprId; + } + + public CatalogRelation getBelongedRelation() { + return belongedRelation; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MappedSlot that = (MappedSlot) o; + return Objects.equals(exprId, that.exprId); + } + + @Override + public int hashCode() { + return Objects.hash(exprId); + } + } + + /** + * Expression and it's index mapping + */ + public static class ExpressionIndexMapping extends Mapping { + private final Multimap expressionIndexMapping; + + public ExpressionIndexMapping(Multimap expressionIndexMapping) { + this.expressionIndexMapping = expressionIndexMapping; + } + + public Multimap getExpressionIndexMapping() { + return expressionIndexMapping; + } + + public static ExpressionIndexMapping generate(List expressions) { + Multimap expressionIndexMapping = ArrayListMultimap.create(); + for (int i = 0; i < expressions.size(); i++) { + expressionIndexMapping.put(expressions.get(i), i); + } + return new ExpressionIndexMapping(expressionIndexMapping); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java new file mode 100644 index 00000000000000..b0de1ccfa469bb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializationContext.java @@ -0,0 +1,68 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.View; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.memo.GroupId; +import org.apache.doris.nereids.trees.plans.Plan; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Maintain the context for query rewrite by materialized view + */ +public class MaterializationContext { + + // TODO add MaterializedView class + private final Plan mvPlan; + private final CascadesContext context; + private final List baseTables; + private final List baseViews; + // Group ids that are rewritten by this mv to reduce rewrite times + private final Set matchedGroups = new HashSet<>(); + private final Plan scanPlan; + + public MaterializationContext(Plan mvPlan, CascadesContext context, + List
baseTables, List baseViews, Plan scanPlan) { + this.mvPlan = mvPlan; + this.context = context; + this.baseTables = baseTables; + this.baseViews = baseViews; + this.scanPlan = scanPlan; + } + + public Set getMatchedGroups() { + return matchedGroups; + } + + public void addMatchedGroup(GroupId groupId) { + matchedGroups.add(groupId); + } + + public Plan getMvPlan() { + return mvPlan; + } + + public Plan getScanPlan() { + return scanPlan; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java new file mode 100644 index 00000000000000..ce9c208e5f5d16 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewAggregateRule.java @@ -0,0 +1,33 @@ +// 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.rewrite.RewriteRuleFactory; + +import java.util.List; + +/** + * This is responsible for aggregate rewriting according to different pattern + * */ +public class MaterializedViewAggregateRule extends AbstractMaterializedViewAggregateRule implements RewriteRuleFactory { + @Override + public List buildRules() { + return null; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java new file mode 100644 index 00000000000000..92f102dc1dece3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewProjectJoinRule.java @@ -0,0 +1,47 @@ +// 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.RulePromise; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; + +import com.google.common.collect.ImmutableList; + +import java.util.List; + +/** + * This is responsible for join rewriting according to different pattern + * */ +public class MaterializedViewProjectJoinRule extends AbstractMaterializedViewJoinRule implements RewriteRuleFactory { + + public static final MaterializedViewProjectJoinRule INSTANCE = new MaterializedViewProjectJoinRule(); + + @Override + public List buildRules() { + return ImmutableList.of( + logicalProject(logicalJoin(any(), any())).thenApplyMulti(ctx -> { + LogicalProject> root = ctx.root; + return rewrite(root, ctx.cascadesContext); + }).toRule(RuleType.MATERIALIZED_VIEW_ONLY_JOIN, RulePromise.EXPLORE)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java new file mode 100644 index 00000000000000..c5909822adbb18 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/MaterializedViewScanRule.java @@ -0,0 +1,34 @@ +// 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.rewrite.RewriteRuleFactory; + +import java.util.List; + +/** + * This is responsible for single table rewriting according to different pattern + * */ +public class MaterializedViewScanRule extends AbstractMaterializedViewRule implements RewriteRuleFactory { + + @Override + public List buildRules() { + return null; + } +} 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 new file mode 100644 index 00000000000000..40b91994be76e9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/Predicates.java @@ -0,0 +1,131 @@ +// 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.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitors.PredicatesSpliter; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Set; + +/** + * This record the predicates which can be pulled up or some other type predicates + * */ +public class Predicates { + + // Predicates that can be pulled up + private final Set pulledUpPredicates; + + public Predicates(Set pulledUpPredicates) { + this.pulledUpPredicates = pulledUpPredicates; + } + + public static Predicates of(Set pulledUpPredicates) { + return new Predicates(pulledUpPredicates); + } + + public Set getPulledUpPredicates() { + return pulledUpPredicates; + } + + public Expression composedExpression() { + return ExpressionUtils.and(pulledUpPredicates); + } + + /** + * Split the expression to equal, range and residual predicate. + * */ + public static SplitPredicate splitPredicates(Expression expression) { + PredicatesSpliter predicatesSplit = new PredicatesSpliter(expression); + expression.accept(predicatesSplit, null); + return predicatesSplit.getSplitPredicate(); + } + + /** + * The split different representation for predicate expression, such as equal, range and residual predicate. + * */ + public static final class SplitPredicate { + private final Expression equalPredicates; + private final Expression rangePredicates; + private final Expression residualPredicates; + + public SplitPredicate(Expression equalPredicates, Expression rangePredicates, Expression residualPredicates) { + this.equalPredicates = equalPredicates; + this.rangePredicates = rangePredicates; + this.residualPredicates = residualPredicates; + } + + public Expression getEqualPredicates() { + return equalPredicates; + } + + public Expression getRangePredicates() { + return rangePredicates; + } + + public Expression getResidualPredicates() { + return residualPredicates; + } + + public static SplitPredicate empty() { + return new SplitPredicate(null, null, null); + } + + /** + * SplitPredicate construct + * */ + public static SplitPredicate of(Expression equalPredicates, + Expression rangePredicates, + Expression residualPredicates) { + return new SplitPredicate(equalPredicates, rangePredicates, residualPredicates); + } + + /** + * isEmpty + * */ + public boolean isEmpty() { + return equalPredicates == null + && rangePredicates == null + && residualPredicates == null; + } + + public Expression composedExpression() { + return ExpressionUtils.and(equalPredicates, rangePredicates, residualPredicates); + } + + public List toList() { + return ImmutableList.of(equalPredicates, rangePredicates, residualPredicates); + } + + /** + * Check the predicates in SplitPredicate is whether all true or not + */ + public boolean isAlwaysTrue() { + return equalPredicates instanceof BooleanLiteral + && rangePredicates instanceof BooleanLiteral + && residualPredicates instanceof BooleanLiteral + && ((BooleanLiteral) equalPredicates).getValue() + && ((BooleanLiteral) rangePredicates).getValue() + && ((BooleanLiteral) residualPredicates).getValue(); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelationMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelationMapping.java new file mode 100644 index 00000000000000..ea91104a246e09 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/RelationMapping.java @@ -0,0 +1,63 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.exploration.mv; + +import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.BiMap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Multimap; + +import java.util.List; + +/** + * Relation mapping + * such as query pattern is a1 left join a2 left join b + * view pattern is a1 left join a2 left join b. the mapping will be + * [{a1:a1, a2:a2, b:b}, {a1:a2, a2:a1, b:b}] + */ +public class RelationMapping extends Mapping { + + private final BiMap mappedRelationMap; + + public RelationMapping(BiMap mappedRelationMap) { + this.mappedRelationMap = mappedRelationMap; + } + + public BiMap getMappedRelationMap() { + return mappedRelationMap; + } + + /** + * Generate mapping according to source and target relation + */ + public static List generate(List source, List target) { + Multimap queryTableRelationIdMap = ArrayListMultimap.create(); + for (CatalogRelation relation : source) { + queryTableRelationIdMap.put(relation.getTable(), relation); + } + Multimap viewTableRelationIdMap = ArrayListMultimap.create(); + for (CatalogRelation relation : target) { + viewTableRelationIdMap.put(relation.getTable(), relation); + } + // todo generate relation map + return ImmutableList.of(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/SlotMapping.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/SlotMapping.java new file mode 100644 index 00000000000000..7c50d79c6a23c2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/SlotMapping.java @@ -0,0 +1,49 @@ +// 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 com.google.common.collect.BiMap; + +/** + * SlotMapping, this is open generated from relationMapping + */ +public class SlotMapping extends Mapping { + + private final BiMap relationSlotMap; + + public SlotMapping(BiMap relationSlotMap) { + this.relationSlotMap = relationSlotMap; + } + + public BiMap getRelationSlotMap() { + return relationSlotMap; + } + + public SlotMapping inverse() { + return SlotMapping.of(relationSlotMap.inverse()); + } + + public static SlotMapping of(BiMap relationSlotMap) { + return new SlotMapping(relationSlotMap); + } + + public static SlotMapping generate(RelationMapping relationMapping) { + // TODO implement + return SlotMapping.of(null); + } +} 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 new file mode 100644 index 00000000000000..141b8a98bccfdb --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/exploration/mv/StructInfo.java @@ -0,0 +1,106 @@ +// 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.jobs.joinorder.hypergraph.HyperGraph; +import org.apache.doris.nereids.memo.Group; +import org.apache.doris.nereids.rules.exploration.mv.Predicates.SplitPredicate; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.util.ExpressionUtils; + +import java.util.List; + +/** + * StructInfo + */ +public class StructInfo { + private final List relations; + private final Predicates predicates; + // Used by predicate compensation + private final EquivalenceClass equivalenceClass; + private final Plan originalPlan; + private final HyperGraph hyperGraph; + + private StructInfo(List relations, + Predicates predicates, + Plan originalPlan, + HyperGraph hyperGraph) { + this.relations = relations; + this.predicates = predicates; + this.originalPlan = originalPlan; + this.hyperGraph = hyperGraph; + // construct equivalenceClass according to equals predicates + this.equivalenceClass = new EquivalenceClass(); + SplitPredicate splitPredicate = Predicates.splitPredicates(predicates.composedExpression()); + for (Expression expression : ExpressionUtils.extractConjunction(splitPredicate.getEqualPredicates())) { + EqualTo equalTo = (EqualTo) expression; + equivalenceClass.addEquivalenceClass( + (SlotReference) equalTo.getArguments().get(0), + (SlotReference) equalTo.getArguments().get(1)); + } + } + + public static StructInfo of(Plan originalPlan) { + // TODO build graph from original plan and get relations and predicates from graph + return new StructInfo(null, null, originalPlan, null); + } + + public static StructInfo of(Group group) { + // TODO build graph from original plan and get relations and predicates from graph + return new StructInfo(null, null, group.getLogicalExpression().getPlan(), null); + } + + public List getRelations() { + return relations; + } + + public Predicates getPredicates() { + return predicates; + } + + public EquivalenceClass getEquivalenceClass() { + return equivalenceClass; + } + + public Plan getOriginalPlan() { + return originalPlan; + } + + public HyperGraph getHyperGraph() { + return hyperGraph; + } + + public List getExpressions() { + return originalPlan instanceof LogicalProject + ? ((LogicalProject) originalPlan).getProjects() : originalPlan.getOutput(); + } + + /** + * Judge the source graph logical is whether the same as target + * For inner join should judge only the join tables, + * for other join type should also judge the join direction, it's input filter that can not be pulled up etc. + * */ + public static boolean isGraphLogicalEquals(HyperGraph source, HyperGraph target) { + return false; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitors.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitors.java index 513da0e93d9112..1d6a82c84a3947 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitors.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitors.java @@ -17,9 +17,16 @@ package org.apache.doris.nereids.trees.expressions.visitor; +import org.apache.doris.nereids.rules.exploration.mv.Predicates; +import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; +import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.util.ExpressionUtils; + +import java.util.ArrayList; +import java.util.List; /** * This is the factory for all ExpressionVisitor instance. @@ -54,4 +61,46 @@ public Boolean visitAggregateFunction(AggregateFunction aggregateFunction, Void return true; } } + + /** + * Split the expression to equal, range and residual predicate. + * Should instance when used. + */ + public static class PredicatesSpliter extends DefaultExpressionVisitor { + + private List equalPredicates = new ArrayList<>(); + private List rangePredicates = new ArrayList<>(); + private List residualPredicates = new ArrayList<>(); + private final Expression target; + + public PredicatesSpliter(Expression target) { + this.target = target; + } + + @Override + public Void visitComparisonPredicate(ComparisonPredicate comparisonPredicate, Void context) { + // TODO Smallest implement, complete later + if (comparisonPredicate instanceof EqualTo) { + Expression argument0 = comparisonPredicate.getArgument(0); + Expression argument1 = comparisonPredicate.getArgument(1); + if (argument0.isSlot() && argument1.isSlot()) { + equalPredicates.add(comparisonPredicate); + } else { + rangePredicates.add(comparisonPredicate); + } + } + return super.visit(comparisonPredicate, context); + } + + public Expression getTarget() { + return target; + } + + public Predicates.SplitPredicate getSplitPredicate() { + return Predicates.SplitPredicate.of( + equalPredicates.isEmpty() ? null : ExpressionUtils.and(equalPredicates), + rangePredicates.isEmpty() ? null : ExpressionUtils.and(rangePredicates), + residualPredicates.isEmpty() ? null : ExpressionUtils.and(residualPredicates)); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java index 0c5faa20957666..00512cb476e50b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/ExpressionUtils.java @@ -17,7 +17,9 @@ package org.apache.doris.nereids.util; +import org.apache.doris.catalog.TableIf.TableType; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.rules.exploration.mv.SlotMapping; import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule; import org.apache.doris.nereids.trees.TreeNode; @@ -39,6 +41,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.plans.Plan; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -196,6 +199,26 @@ public static Expression combine(Class type, Collection shuttleExpressionWithLineage(List expression, + Plan plan, + Set targetTypes, + Set tableIdentifiers) { + return ImmutableList.of(); + } + + // Replace the slot in expressions according to the slotMapping + // if any slot cannot be mapped then return null + public static List permute(List expressions, SlotMapping slotMapping) { + return ImmutableList.of(); + } + /** * Choose the minimum slot from input parameter. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 3cc5bd455874c0..a706c6ba3a6bb8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -449,6 +449,12 @@ public class SessionVariable implements Serializable, Writable { public static final String TABLE_STATS_HEALTH_THRESHOLD = "table_stats_health_threshold"; + public static final String ENABLE_MATERIALIZED_VIEW_REWRITE + = "enable_materialized_view_rewrite"; + + public static final String MATERIALIZED_VIEW_REWRITE_ENABLE_CONTAIN_FOREIGN_TABLE + = "materialized_view_rewrite_enable_contain_foreign_table"; + public static final List DEBUG_VARIABLES = ImmutableList.of( SKIP_DELETE_PREDICATE, SKIP_DELETE_BITMAP, @@ -1359,6 +1365,17 @@ public void setEnableLeftZigZag(boolean enableLeftZigZag) { + "considered outdated."}) public int tableStatsHealthThreshold = 60; + @VariableMgr.VarAttr(name = ENABLE_MATERIALIZED_VIEW_REWRITE, needForward = true, + description = {"是否开启基于结构信息的透明改写", + "Whether to enable materialized rewriting based on struct info"}) + public boolean enableMaterializedViewRewrite = false; + + @VariableMgr.VarAttr(name = MATERIALIZED_VIEW_REWRITE_ENABLE_CONTAIN_FOREIGN_TABLE, needForward = true, + description = {"基于结构信息的透明改写,是否使用包含外表的物化视图", + "whether to use a materialized view that contains the foreign table" + + "for rewriting based on struct info"}) + public boolean materializedViewRewriteEnableContainForeignTable = false; + public static final String IGNORE_RUNTIME_FILTER_IDS = "ignore_runtime_filter_ids"; public Set getIgnoredRuntimeFilterIds() { @@ -2928,4 +2945,12 @@ public void checkSqlDialect(String sqlDialect) { public boolean isEnableInsertGroupCommit() { return enableInsertGroupCommit || Config.wait_internal_group_commit_finish; } + + public boolean isEnableMaterializedViewRewrite() { + return enableMaterializedViewRewrite; + } + + public boolean isMaterializedViewRewriteEnableContainForeignTable() { + return materializedViewRewriteEnableContainForeignTable; + } }