Skip to content

Commit

Permalink
[fix](Nereids): slot set in condition can be empty (#32169)
Browse files Browse the repository at this point in the history
(cherry picked from commit ca09213)
  • Loading branch information
jackwener committed Mar 21, 2024
1 parent de52687 commit 88959f6
Show file tree
Hide file tree
Showing 6 changed files with 174 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,8 @@
import org.apache.doris.nereids.rules.rewrite.PruneOlapScanTablet;
import org.apache.doris.nereids.rules.rewrite.PullUpCteAnchor;
import org.apache.doris.nereids.rules.rewrite.PullUpProjectUnderApply;
import org.apache.doris.nereids.rules.rewrite.PullUpProjectUnderLimit;
import org.apache.doris.nereids.rules.rewrite.PullUpProjectUnderTopN;
import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoEsScan;
import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoJdbcScan;
import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoOdbcScan;
Expand Down Expand Up @@ -321,6 +323,10 @@ public class Rewriter extends AbstractBatchJobExecutor {
new PushdownLimitDistinctThroughJoin(),
new PushdownTopNThroughWindow(),
new CreatePartitionTopNFromWindow()
),
topDown(
new PullUpProjectUnderTopN(),
new PullUpProjectUnderLimit()
)
),
// TODO: these rules should be implementation rules, and generate alternative physical plans.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,6 +260,8 @@ public enum RuleType {
PUSH_TOP_N_THROUGH_PROJECT_JOIN(RuleTypeClass.REWRITE),
PUSH_TOP_N_THROUGH_PROJECT_WINDOW(RuleTypeClass.REWRITE),
PUSH_TOP_N_THROUGH_WINDOW(RuleTypeClass.REWRITE),
PULL_UP_PROJECT_UNDER_TOPN(RuleTypeClass.REWRITE),
PULL_UP_PROJECT_UNDER_LIMIT(RuleTypeClass.REWRITE),
// limit distinct push down
PUSH_LIMIT_DISTINCT_THROUGH_JOIN(RuleTypeClass.REWRITE),
PUSH_LIMIT_DISTINCT_THROUGH_PROJECT_JOIN(RuleTypeClass.REWRITE),
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
// 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;

import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.expressions.Slot;
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 org.apache.doris.nereids.util.PlanUtils;

import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;

import java.util.Set;
import java.util.stream.Collectors;

/**
* Pull up Project under Limit.
*/
public class PullUpProjectUnderLimit extends OneRewriteRuleFactory {
@Override
public Rule build() {
return logicalLimit(logicalProject(logicalJoin().when(j -> j.getJoinType().isLeftRightOuterOrCrossJoin()))
.whenNot(p -> p.isAllSlots()))
.then(limit -> {
LogicalProject<LogicalJoin<Plan, Plan>> project = limit.child();
Set<Slot> allUsedSlots = project.getProjects().stream().flatMap(ne -> ne.getInputSlots().stream())
.collect(Collectors.toSet());
Set<Slot> outputSet = project.child().getOutputSet();
if (outputSet.size() == allUsedSlots.size()) {
Preconditions.checkState(outputSet.equals(allUsedSlots));
return project.withChildren(limit.withChildren(project.child()));
} else {
Plan columnProject = PlanUtils.projectOrSelf(ImmutableList.copyOf(allUsedSlots),
project.child());
return project.withChildren(limit.withChildren(columnProject));
}
}).toRule(RuleType.PULL_UP_PROJECT_UNDER_LIMIT);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
// 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;

import org.apache.doris.nereids.properties.OrderKey;
import org.apache.doris.nereids.rules.Rule;
import org.apache.doris.nereids.rules.RuleType;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Slot;
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 org.apache.doris.nereids.util.ExpressionUtils;

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

/**
* Pull up Project under TopN for PushDownTopNThroughJoin
*/
public class PullUpProjectUnderTopN extends OneRewriteRuleFactory {
@Override
public Rule build() {
return logicalTopN(logicalProject(logicalJoin().when(j -> j.getJoinType().isLeftRightOuterOrCrossJoin()))
.whenNot(p -> p.isAllSlots()))
.then(topN -> {
LogicalProject<LogicalJoin<Plan, Plan>> project = topN.child();
LogicalJoin<Plan, Plan> join = project.child();
// Set<Slot> outputSet = join.getOutputSet();

Plan newTopN;
Map<Slot, Expression> slotMap = ExpressionUtils.generateReplaceMap(project.getProjects());
List<OrderKey> newOrderKeys = topN.getOrderKeys().stream().map(orderKey -> {
Expression expr = orderKey.getExpr();
if (expr instanceof Slot && slotMap.containsKey((Slot) expr)) {
return orderKey.withExpression(ExpressionUtils.replace(expr, slotMap));
} else {
return orderKey;
}
}).collect(Collectors.toList());
if (!newOrderKeys.equals(topN.getOrderKeys())) {
newTopN = topN.withOrderKeys(newOrderKeys);
} else {
newTopN = topN;
}

// Set<Slot> allUsedSlots = project.getProjects().stream().flatMap(ne -> ne.getInputSlots().stream())
// .collect(Collectors.toSet());
// if (outputSet.size() == allUsedSlots.size()) {
// Preconditions.checkState(outputSet.equals(allUsedSlots));
return project.withChildren(newTopN.withChildren(join));
// } else {
// Plan columnProject = PlanUtils.projectOrSelf(ImmutableList.copyOf(allUsedSlots),
// join);
// return project.withChildren(newTopN.withChildren(columnProject));
// }
}).toRule(RuleType.PULL_UP_PROJECT_UNDER_TOPN);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,10 @@ public final boolean isRightOuterJoin() {
return this == RIGHT_OUTER_JOIN;
}

public final boolean isLeftRightOuterOrCrossJoin() {
return this == LEFT_OUTER_JOIN || this == RIGHT_OUTER_JOIN || this == CROSS_JOIN;
}

public final boolean isLeftSemiOrAntiJoin() {
return this == LEFT_SEMI_JOIN || this == LEFT_ANTI_JOIN || this == NULL_AWARE_LEFT_ANTI_JOIN;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext;
import org.apache.doris.nereids.rules.expression.rules.FoldConstantRule;
import org.apache.doris.nereids.trees.TreeNode;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.ComparisonPredicate;
Expand All @@ -44,6 +45,7 @@
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
Expand Down Expand Up @@ -252,6 +254,34 @@ public static Optional<Slot> extractSlotOrCastOnSlot(Expression expr) {
}
}

/**
* Generate replaceMap Slot -> Expression from NamedExpression[Expression as name]
*/
public static Map<Slot, Expression> generateReplaceMap(List<NamedExpression> namedExpressions) {
ImmutableMap.Builder<Slot, Expression> replaceMap = ImmutableMap.builderWithExpectedSize(
namedExpressions.size() * 2);
for (NamedExpression namedExpression : namedExpressions) {
if (namedExpression instanceof Alias) {
// Avoid cast to alias, retrieving the first child expression.
replaceMap.put(namedExpression.toSlot(), namedExpression.child(0));
}
}
return replaceMap.build();
}

/**
* replace NameExpression.
*/
public static NamedExpression replaceNameExpression(NamedExpression expr,
Map<? extends Expression, ? extends Expression> replaceMap) {
Expression newExpr = replace(expr, replaceMap);
if (newExpr instanceof NamedExpression) {
return (NamedExpression) newExpr;
} else {
return new Alias(expr.getExprId(), newExpr, expr.getName());
}
}

/**
* Replace expression node in the expression tree by `replaceMap` in top-down manner.
* For example.
Expand Down

0 comments on commit 88959f6

Please sign in to comment.