Skip to content

Commit

Permalink
[planner](poc)Poc
Browse files Browse the repository at this point in the history
  • Loading branch information
seawinde committed Nov 13, 2023
1 parent 8160a04 commit c2a377b
Show file tree
Hide file tree
Showing 34 changed files with 1,828 additions and 41 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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.rewrite.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;
Expand Down Expand Up @@ -112,6 +113,8 @@ public class CascadesContext implements ScheduleContext {
private final Optional<CTEId> currentTree;
private final Optional<CascadesContext> parent;

private List<MaterializationContext> materializationContexts;

/**
* Constructor of OptimizerContext.
*
Expand All @@ -133,6 +136,7 @@ private CascadesContext(Optional<CascadesContext> parent, Optional<CTEId> curren
this.currentJobContext = new JobContext(this, requireProperties, Double.MAX_VALUE);
this.subqueryExprIsAnalyzed = new HashMap<>();
this.runtimeFilterContext = new RuntimeFilterContext(getConnectContext().getSessionVariable());
this.materializationContexts = new ArrayList<>();
}

/**
Expand Down Expand Up @@ -309,6 +313,14 @@ public void setOuterScope(@Nullable Scope outerScope) {
this.outerScope = Optional.ofNullable(outerScope);
}

public List<MaterializationContext> getMaterializationContexts() {
return materializationContexts;
}

public void addMaterializationContext(MaterializationContext materializationContext) {
this.materializationContexts.add(materializationContext);
}

/**
* getAndCacheSessionVariable
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -243,9 +243,12 @@ private LogicalPlan preprocess(LogicalPlan logicalPlan) {
}

private void initCascadesContext(LogicalPlan plan, PhysicalProperties requireProperties) {
cascadesContext = CascadesContext.initContext(statementContext, plan, requireProperties);
if (statementContext.getConnectContext().getTables() != null) {
cascadesContext.setTables(statementContext.getConnectContext().getTables());
// should call statementContext.getConnectContext().getEnv().getMgr
if (cascadesContext == null) {
cascadesContext = CascadesContext.initContext(statementContext, plan, requireProperties);
if (statementContext.getConnectContext().getTables() != null) {
cascadesContext.setTables(statementContext.getConnectContext().getTables());
}
}
}

Expand Down Expand Up @@ -463,6 +466,11 @@ public CascadesContext getCascadesContext() {
return cascadesContext;
}

@VisibleForTesting
public void setCascadesContext(CascadesContext cascadesContext) {
this.cascadesContext = cascadesContext;
}

public static PhysicalProperties buildInitRequireProperties() {
return PhysicalProperties.GATHER;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@
import org.apache.doris.nereids.memo.GroupExpression;
import org.apache.doris.nereids.rules.Rule;

import java.util.Collections;
import java.util.ArrayList;
import java.util.List;

/**
Expand Down Expand Up @@ -76,22 +76,26 @@ private List<Rule> getExplorationRules() {
|| (groupExpression.getOwnerGroup() != null && !groupExpression.getOwnerGroup().isStatsReliable());
int joinNumBushyTree = context.getCascadesContext().getConnectContext()
.getSessionVariable().getMaxJoinNumBushyTree();

List<Rule> exploreRules = new ArrayList<>(getRuleSet().getMaterializedViewRules());

if (isDisableJoinReorder) {
return Collections.emptyList();
//todo
} else if (isDpHyp) {
if (isOtherJoinReorder) {
return getRuleSet().getDPHypReorderRules();
exploreRules.addAll(getRuleSet().getDPHypReorderRules());
} else {
return Collections.emptyList();
//todo
}
} else if (isLeftZigZagTree) {
return getRuleSet().getLeftZigZagTreeJoinReorder();
} else if (isEnableBushyTree) {
return getRuleSet().getBushyTreeJoinReorder();
exploreRules.addAll(getRuleSet().getBushyTreeJoinReorder());
} else if (context.getCascadesContext().getStatementContext().getMaxNAryInnerJoin() <= joinNumBushyTree) {
return getRuleSet().getBushyTreeJoinReorder();
exploreRules.addAll(getRuleSet().getBushyTreeJoinReorder());
} else {
return getRuleSet().getZigZagTreeJoinReorder();
exploreRules.addAll(getRuleSet().getZigZagTreeJoinReorder());
}
return exploreRules;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,11 @@

/**
* HyperGraph Node.
* Jc
* \
* F
* \
* JC
*/
public class Node {
private final int index;
Expand Down
12 changes: 12 additions & 0 deletions fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
Original file line number Diff line number Diff line change
Expand Up @@ -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.rewrite.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;
Expand Down Expand Up @@ -74,6 +75,8 @@ public class Group {

private int chosenGroupExpressionId = -1;

private Optional<StructInfo> structInfo = Optional.empty();

/**
* Constructor for Group.
*
Expand Down Expand Up @@ -152,6 +155,7 @@ public GroupExpression logicalExpressionsAt(int index) {
* @return the first logical group expression in this group
*/
public GroupExpression getLogicalExpression() {
// poc tmp
Preconditions.checkArgument(logicalExpressions.size() == 1,
"There should be only one Logical Expression in Group");
return logicalExpressions.get(0);
Expand Down Expand Up @@ -532,4 +536,12 @@ public String treeString() {

return TreeStringUtils.treeString(this, toString, getChildren, getExtraPlans, displayExtraPlan);
}

public Optional<StructInfo> getStructInfo() {
return structInfo;
}

public void setStructInfo(StructInfo structInfo) {
this.structInfo = Optional.ofNullable(structInfo);
}
}
17 changes: 17 additions & 0 deletions fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
Original file line number Diff line number Diff line change
Expand Up @@ -314,6 +314,7 @@ private Group init(Plan plan) {
plan = replaceChildrenToGroupPlan(plan, childrenGroups);
GroupExpression newGroupExpression = new GroupExpression(plan, childrenGroups);
Group group = new Group(groupIdGenerator.getNextId(), newGroupExpression, plan.getLogicalProperties());
// PoC add struct info to group

groups.put(group.getGroupId(), group);
if (groupExpressions.containsKey(newGroupExpression)) {
Expand All @@ -323,6 +324,22 @@ private Group init(Plan plan) {
return group;
}

/** initPoC */
public Group initPoC(Plan plan) {
Preconditions.checkArgument(!(plan instanceof GroupPlan), "Cannot init memo by a GroupPlan");

/* initialize children recursively */
List<Group> childrenGroups = new ArrayList<>(plan.arity());
for (Plan child : plan.children()) {
childrenGroups.add(initPoC(child));
}

plan = replaceChildrenToGroupPlan(plan, childrenGroups);
GroupExpression newGroupExpression = new GroupExpression(plan, childrenGroups);
Group group = new Group(groupIdGenerator.getNextId(), newGroupExpression, plan.getLogicalProperties());
return group;
}

/**
* add or replace the plan into the target group.
* <p>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@
import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughWindow;
import org.apache.doris.nereids.rules.rewrite.PushdownJoinOtherCondition;
import org.apache.doris.nereids.rules.rewrite.PushdownProjectThroughLimit;
import org.apache.doris.nereids.rules.rewrite.mv.MaterializedViewProjectJoinRule;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
Expand Down Expand Up @@ -220,6 +221,10 @@ public class RuleSet {
.add(JoinCommute.BUSHY.build())
.build();

public static final List<Rule> MATERIALIZED_VIEW_RULES = planRuleFactories()
.add(MaterializedViewProjectJoinRule.INSTANCE)
.build();

public List<Rule> getDPHypReorderRules() {
return DPHYP_REORDER_RULES;
}
Expand All @@ -240,6 +245,10 @@ public List<Rule> getImplementationRules() {
return IMPLEMENTATION_RULES;
}

public List<Rule> getMaterializedViewRules() {
return MATERIALIZED_VIEW_RULES;
}

public static RuleFactories planRuleFactories() {
return new RuleFactories();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ private Plan deferMaterialize(LogicalResultSink<? extends Plan> logicalResultSin
LogicalTopN<? extends Plan> logicalTopN, Optional<LogicalFilter<? extends Plan>> logicalFilter,
LogicalOlapScan logicalOlapScan) {
Column rowId = new Column(Column.ROWID_COL, Type.STRING, false, null, false, "", "rowid column");
SlotReference columnId = SlotReference.fromColumn(rowId, logicalOlapScan.getQualifier());
SlotReference columnId = SlotReference.fromColumn(rowId, logicalOlapScan.getQualifier(), null);
Set<ExprId> deferredMaterializedExprIds = Sets.newHashSet(logicalOlapScan.getOutputExprIdSet());
logicalFilter.ifPresent(filter -> filter.getConjuncts()
.forEach(e -> deferredMaterializedExprIds.removeAll(e.getInputSlotExprIds())));
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
// 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.rewrite.mv;

/**
* AbstractMaterializedViewAggregateRule
* */
public abstract class AbstractMaterializedViewAggregateRule extends AbstractMaterializedViewRule {
}
Loading

0 comments on commit c2a377b

Please sign in to comment.