[feature](Nereids): pushdown MIN/MAX/SUM through join (#22264)

* [minor](Nereids): add more comment to explain code

* [feature](Nereids): pushdown MIN/MAX/SUM through join
This commit is contained in:
jakevin
2023-08-01 13:23:55 +08:00
committed by GitHub
parent 3a11de889f
commit 26737dddff
13 changed files with 617 additions and 13 deletions

View File

@ -29,7 +29,7 @@ import java.util.List;
*/
public class Node {
private final int index;
// TODO
// Due to group in Node is base group, so mergeGroup() don't need to consider it.
private final Group group;
private final List<Edge> edges = new ArrayList<>();

View File

@ -319,6 +319,10 @@ public class PlanReceiver implements AbstractReceiver {
hasGenerated.add(groupExpression);
// process child first, plan's child may be changed due to mergeGroup
// due to mergeGroup, the children Group of groupExpression may be replaced, so we need to use lambda to
// get the child to make we can get child at the time we use child.
// If we use for child: groupExpression.children(), it means that we take it in advance. It may cause NPE,
// work flow: get children() to get left, right -> copyIn left() -> mergeGroup -> right is merged -> NPE
Plan physicalPlan = groupExpression.getPlan();
for (int i = 0; i < groupExpression.children().size(); i++) {
int childIdx = i;

View File

@ -149,12 +149,15 @@ public enum RuleType {
PUSHDOWN_FILTER_THROUGH_CTE(RuleTypeClass.REWRITE),
PUSHDOWN_FILTER_THROUGH_CTE_ANCHOR(RuleTypeClass.REWRITE),
PUSH_DOWN_DISTINCT_THROUGH_JOIN(RuleTypeClass.REWRITE),
PUSHDOWN_DISTINCT_THROUGH_JOIN(RuleTypeClass.REWRITE),
COLUMN_PRUNING(RuleTypeClass.REWRITE),
PUSHDOWN_TOP_N_THROUGH_PROJECTION_WINDOW(RuleTypeClass.REWRITE),
PUSHDOWN_TOP_N_THROUGH_WINDOW(RuleTypeClass.REWRITE),
PUSHDOWN_MIN_MAX_THROUGH_JOIN(RuleTypeClass.REWRITE),
PUSHDOWN_SUM_THROUGH_JOIN(RuleTypeClass.REWRITE),
PUSHDOWN_COUNT_THROUGH_JOIN(RuleTypeClass.REWRITE),
TRANSPOSE_LOGICAL_SEMI_JOIN_LOGICAL_JOIN(RuleTypeClass.REWRITE),
TRANSPOSE_LOGICAL_SEMI_JOIN_LOGICAL_JOIN_PROJECT(RuleTypeClass.REWRITE),
@ -269,8 +272,8 @@ public enum RuleType {
TRANSPOSE_LOGICAL_AGG_SEMI_JOIN(RuleTypeClass.EXPLORATION),
TRANSPOSE_LOGICAL_AGG_SEMI_JOIN_PROJECT(RuleTypeClass.EXPLORATION),
TRANSPOSE_LOGICAL_JOIN_UNION(RuleTypeClass.EXPLORATION),
PUSH_DOWN_PROJECT_THROUGH_SEMI_JOIN(RuleTypeClass.EXPLORATION),
PUSH_DOWN_PROJECT_THROUGH_INNER_OUTER_JOIN(RuleTypeClass.EXPLORATION),
PUSHDOWN_PROJECT_THROUGH_SEMI_JOIN(RuleTypeClass.EXPLORATION),
PUSHDOWN_PROJECT_THROUGH_INNER_OUTER_JOIN(RuleTypeClass.EXPLORATION),
EAGER_COUNT(RuleTypeClass.EXPLORATION),
EAGER_GROUP_BY(RuleTypeClass.EXPLORATION),
EAGER_GROUP_BY_COUNT(RuleTypeClass.EXPLORATION),

View File

@ -78,7 +78,7 @@ public class EagerGroupByCount extends OneExplorationRuleFactory {
rightSums.add(sum);
}
}
if (leftSums.size() == 0 || rightSums.size() == 0) {
if (leftSums.size() == 0 && rightSums.size() == 0) {
return null;
}

View File

@ -80,7 +80,7 @@ public class EagerSplit extends OneExplorationRuleFactory {
rightSums.add(sum);
}
}
if (leftSums.size() == 0 || rightSums.size() == 0) {
if (leftSums.size() == 0 && rightSums.size() == 0) {
return null;
}

View File

@ -69,7 +69,7 @@ public class PushdownProjectThroughInnerOuterJoin implements ExplorationRuleFact
return null;
}
return topJoin.withChildren(newLeft, topJoin.right());
}).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_INNER_OUTER_JOIN),
}).toRule(RuleType.PUSHDOWN_PROJECT_THROUGH_INNER_OUTER_JOIN),
logicalJoin(group(), logicalProject(logicalJoin()))
.when(j -> j.right().child().getJoinType().isOuterJoin()
|| j.right().child().getJoinType().isInnerJoin())
@ -83,7 +83,7 @@ public class PushdownProjectThroughInnerOuterJoin implements ExplorationRuleFact
return null;
}
return topJoin.withChildren(topJoin.left(), newRight);
}).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_INNER_OUTER_JOIN)
}).toRule(RuleType.PUSHDOWN_PROJECT_THROUGH_INNER_OUTER_JOIN)
);
}

View File

@ -63,7 +63,7 @@ public class PushdownProjectThroughSemiJoin implements ExplorationRuleFactory {
LogicalProject<LogicalJoin<GroupPlan, GroupPlan>> project = topJoin.left();
Plan newLeft = pushdownProject(project);
return topJoin.withChildren(newLeft, topJoin.right());
}).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_SEMI_JOIN),
}).toRule(RuleType.PUSHDOWN_PROJECT_THROUGH_SEMI_JOIN),
logicalJoin(group(), logicalProject(logicalJoin()))
.when(j -> j.right().child().getJoinType().isLeftSemiOrAntiJoin())
@ -74,7 +74,7 @@ public class PushdownProjectThroughSemiJoin implements ExplorationRuleFactory {
LogicalProject<LogicalJoin<GroupPlan, GroupPlan>> project = topJoin.right();
Plan newRight = pushdownProject(project);
return topJoin.withChildren(topJoin.left(), newRight);
}).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_SEMI_JOIN)
}).toRule(RuleType.PUSHDOWN_PROJECT_THROUGH_SEMI_JOIN)
);
}

View File

@ -0,0 +1,181 @@
// 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.Alias;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Related paper "Eager aggregation and lazy aggregation".
* <pre>
* aggregate: Min/Max(x)
* |
* join
* | \
* | *
* (x)
* ->
* aggregate: Min/Max(min1)
* |
* join
* | \
* | *
* aggregate: Min/Max(x) as min1
* </pre>
*/
public class PushdownMinMaxThroughJoin implements RewriteRuleFactory {
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalAggregate(innerLogicalJoin())
.when(agg -> agg.child().getOtherJoinConjuncts().size() == 0)
.whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate))
.when(agg -> {
Set<AggregateFunction> funcs = agg.getAggregateFunctions();
return !funcs.isEmpty() && funcs.stream()
.allMatch(f -> (f instanceof Min || f instanceof Max) && f.child(0) instanceof Slot);
})
.then(agg -> pushMinMax(agg, agg.child(), ImmutableList.of()))
.toRule(RuleType.PUSHDOWN_MIN_MAX_THROUGH_JOIN),
logicalAggregate(logicalProject(innerLogicalJoin()))
.when(agg -> agg.child().isAllSlots())
.when(agg -> agg.child().child().getOtherJoinConjuncts().size() == 0)
.whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate))
.when(agg -> {
Set<AggregateFunction> funcs = agg.getAggregateFunctions();
return !funcs.isEmpty() && funcs.stream()
.allMatch(f -> (f instanceof Min || f instanceof Max) && f.child(0) instanceof Slot);
})
.then(agg -> pushMinMax(agg, agg.child().child(), agg.child().getProjects()))
.toRule(RuleType.PUSHDOWN_MIN_MAX_THROUGH_JOIN)
);
}
private LogicalAggregate<Plan> pushMinMax(LogicalAggregate<? extends Plan> agg,
LogicalJoin<Plan, Plan> join, List<NamedExpression> projects) {
List<Slot> leftOutput = join.left().getOutput();
List<Slot> rightOutput = join.right().getOutput();
List<AggregateFunction> leftFuncs = new ArrayList<>();
List<AggregateFunction> rightFuncs = new ArrayList<>();
for (AggregateFunction func : agg.getAggregateFunctions()) {
Slot slot = (Slot) func.child(0);
if (leftOutput.contains(slot)) {
leftFuncs.add(func);
} else if (rightOutput.contains(slot)) {
rightFuncs.add(func);
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
}
Set<Slot> leftGroupBy = new HashSet<>();
Set<Slot> rightGroupBy = new HashSet<>();
for (Expression e : agg.getGroupByExpressions()) {
Slot slot = (Slot) e;
if (leftOutput.contains(slot)) {
leftGroupBy.add(slot);
} else if (rightOutput.contains(slot)) {
rightGroupBy.add(slot);
} else {
return null;
}
}
join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> {
if (leftOutput.contains(slot)) {
leftGroupBy.add(slot);
} else if (rightOutput.contains(slot)) {
rightGroupBy.add(slot);
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
}));
Plan left = join.left();
Plan right = join.right();
Map<Slot, NamedExpression> leftSlotToOutput = new HashMap<>();
Map<Slot, NamedExpression> rightSlotToOutput = new HashMap<>();
if (!leftFuncs.isEmpty()) {
Builder<NamedExpression> leftAggOutputBuilder = ImmutableList.<NamedExpression>builder()
.addAll(leftGroupBy);
leftFuncs.forEach(func -> {
Alias alias = func.alias(func.getName());
leftSlotToOutput.put((Slot) func.child(0), alias);
leftAggOutputBuilder.add(alias);
});
left = new LogicalAggregate<>(ImmutableList.copyOf(leftGroupBy), leftAggOutputBuilder.build(), join.left());
}
if (!rightFuncs.isEmpty()) {
Builder<NamedExpression> rightAggOutputBuilder = ImmutableList.<NamedExpression>builder()
.addAll(rightGroupBy);
rightFuncs.forEach(func -> {
Alias alias = func.alias(func.getName());
rightSlotToOutput.put((Slot) func.child(0), alias);
rightAggOutputBuilder.add(alias);
});
right = new LogicalAggregate<>(ImmutableList.copyOf(rightGroupBy), rightAggOutputBuilder.build(),
join.right());
}
Preconditions.checkState(left != join.left() || right != join.right());
Plan newJoin = join.withChildren(left, right);
List<NamedExpression> newOutputExprs = new ArrayList<>();
for (NamedExpression ne : agg.getOutputExpressions()) {
if (ne instanceof Alias && ((Alias) ne).child() instanceof AggregateFunction) {
AggregateFunction func = (AggregateFunction) ((Alias) ne).child();
Slot slot = (Slot) func.child(0);
if (leftSlotToOutput.containsKey(slot)) {
Expression newFunc = func.withChildren(leftSlotToOutput.get(slot).toSlot());
newOutputExprs.add((NamedExpression) ne.withChildren(newFunc));
} else if (rightSlotToOutput.containsKey(slot)) {
Expression newFunc = func.withChildren(rightSlotToOutput.get(slot).toSlot());
newOutputExprs.add((NamedExpression) ne.withChildren(newFunc));
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
} else {
newOutputExprs.add(ne);
}
}
// TODO: column prune project
return agg.withAggOutputChild(newOutputExprs, newJoin);
}
}

View File

@ -0,0 +1,194 @@
// 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.Alias;
import org.apache.doris.nereids.trees.expressions.Expression;
import org.apache.doris.nereids.trees.expressions.Multiply;
import org.apache.doris.nereids.trees.expressions.NamedExpression;
import org.apache.doris.nereids.trees.expressions.Slot;
import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction;
import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
import org.apache.doris.nereids.trees.plans.logical.LogicalJoin;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableList.Builder;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Related paper "Eager aggregation and lazy aggregation".
* <pre>
* aggregate: Sum(x)
* |
* join
* | \
* | *
* (x)
* ->
* aggregate: Sum(min1)
* |
* join
* | \
* | *
* aggregate: Sum(x) as min1
* </pre>
*/
public class PushdownSumThroughJoin implements RewriteRuleFactory {
@Override
public List<Rule> buildRules() {
return ImmutableList.of(
logicalAggregate(innerLogicalJoin())
.when(agg -> agg.child().getOtherJoinConjuncts().size() == 0)
.whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate))
.when(agg -> {
Set<AggregateFunction> funcs = agg.getAggregateFunctions();
return !funcs.isEmpty() && funcs.stream()
.allMatch(f -> f instanceof Sum && f.child(0) instanceof Slot);
})
.then(agg -> pushSum(agg, agg.child(), ImmutableList.of()))
.toRule(RuleType.PUSHDOWN_SUM_THROUGH_JOIN),
logicalAggregate(logicalProject(innerLogicalJoin()))
.when(agg -> agg.child().isAllSlots())
.when(agg -> agg.child().child().getOtherJoinConjuncts().size() == 0)
.whenNot(agg -> agg.child().children().stream().anyMatch(p -> p instanceof LogicalAggregate))
.when(agg -> {
Set<AggregateFunction> funcs = agg.getAggregateFunctions();
return !funcs.isEmpty() && funcs.stream()
.allMatch(f -> f instanceof Sum && f.child(0) instanceof Slot);
})
.then(agg -> pushSum(agg, agg.child().child(), agg.child().getProjects()))
.toRule(RuleType.PUSHDOWN_SUM_THROUGH_JOIN)
);
}
private LogicalAggregate<Plan> pushSum(LogicalAggregate<? extends Plan> agg,
LogicalJoin<Plan, Plan> join, List<NamedExpression> projects) {
List<Slot> leftOutput = join.left().getOutput();
List<Slot> rightOutput = join.right().getOutput();
List<Sum> leftSums = new ArrayList<>();
List<Sum> rightSums = new ArrayList<>();
for (AggregateFunction f : agg.getAggregateFunctions()) {
Sum sum = (Sum) f;
Slot slot = (Slot) sum.child();
if (leftOutput.contains(slot)) {
leftSums.add(sum);
} else if (rightOutput.contains(slot)) {
rightSums.add(sum);
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
}
if (leftSums.isEmpty() && rightSums.isEmpty()
|| (!leftSums.isEmpty() && !rightSums.isEmpty())) {
return null;
}
Set<Slot> leftGroupBy = new HashSet<>();
Set<Slot> rightGroupBy = new HashSet<>();
for (Expression e : agg.getGroupByExpressions()) {
Slot slot = (Slot) e;
if (leftOutput.contains(slot)) {
leftGroupBy.add(slot);
} else if (rightOutput.contains(slot)) {
rightGroupBy.add(slot);
} else {
return null;
}
}
join.getHashJoinConjuncts().forEach(e -> e.getInputSlots().forEach(slot -> {
if (leftOutput.contains(slot)) {
leftGroupBy.add(slot);
} else if (rightOutput.contains(slot)) {
rightGroupBy.add(slot);
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
}));
List<Sum> sums;
Set<Slot> sumGroupBy;
Set<Slot> cntGroupBy;
Plan sumChild;
Plan cntChild;
if (!leftSums.isEmpty()) {
sums = leftSums;
sumGroupBy = leftGroupBy;
cntGroupBy = rightGroupBy;
sumChild = join.left();
cntChild = join.right();
} else {
sums = rightSums;
sumGroupBy = rightGroupBy;
cntGroupBy = leftGroupBy;
sumChild = join.right();
cntChild = join.left();
}
// Sum agg
Map<Slot, NamedExpression> sumSlotToOutput = new HashMap<>();
Builder<NamedExpression> sumAggOutputBuilder = ImmutableList.<NamedExpression>builder().addAll(sumGroupBy);
sums.forEach(func -> {
Alias alias = func.alias(func.getName());
sumSlotToOutput.put((Slot) func.child(0), alias);
sumAggOutputBuilder.add(alias);
});
LogicalAggregate<Plan> sumAgg = new LogicalAggregate<>(
ImmutableList.copyOf(sumGroupBy), sumAggOutputBuilder.build(), sumChild);
// Count agg
Alias cnt = new Count().alias("cnt");
List<NamedExpression> cntAggOutput = ImmutableList.<NamedExpression>builder()
.addAll(cntGroupBy).add(cnt)
.build();
LogicalAggregate<Plan> cntAgg = new LogicalAggregate<>(
ImmutableList.copyOf(cntGroupBy), cntAggOutput, cntChild);
Plan newJoin = !leftSums.isEmpty() ? join.withChildren(sumAgg, cntAgg) : join.withChildren(cntAgg, sumAgg);
// top Sum agg
// replace sum(x) -> sum(sum# * cnt)
List<NamedExpression> newOutputExprs = new ArrayList<>();
for (NamedExpression ne : agg.getOutputExpressions()) {
if (ne instanceof Alias && ((Alias) ne).child() instanceof AggregateFunction) {
AggregateFunction func = (AggregateFunction) ((Alias) ne).child();
Slot slot = (Slot) func.child(0);
if (sumSlotToOutput.containsKey(slot)) {
Expression expr = func.withChildren(new Multiply(sumSlotToOutput.get(slot).toSlot(), cnt.toSlot()));
newOutputExprs.add((NamedExpression) ne.withChildren(expr));
} else {
throw new IllegalStateException("Slot " + slot + " not found in join output");
}
} else {
newOutputExprs.add(ne);
}
}
return agg.withAggOutputChild(newOutputExprs, newJoin);
}
}

View File

@ -111,10 +111,9 @@ public class Statistics {
ColumnStatistic columnStatistic = entry.getValue();
ColumnStatisticBuilder columnStatisticBuilder = new ColumnStatisticBuilder(columnStatistic);
columnStatisticBuilder.setNdv(Math.min(columnStatistic.ndv, rowCount));
double nullFactor = (rowCount - columnStatistic.numNulls) / rowCount;
columnStatisticBuilder.setNumNulls(nullFactor * rowCount);
columnStatisticBuilder.setNumNulls(rowCount - columnStatistic.numNulls);
columnStatisticBuilder.setCount(rowCount);
statistics.addColumnStats(entry.getKey(), columnStatisticBuilder.build());
expressionToColumnStats.put(entry.getKey(), columnStatisticBuilder.build());
}
return statistics;
}

View File

@ -0,0 +1,114 @@
// 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.common.Pair;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.functions.agg.Max;
import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.util.LogicalPlanBuilder;
import org.apache.doris.nereids.util.MemoPatternMatchSupported;
import org.apache.doris.nereids.util.MemoTestUtils;
import org.apache.doris.nereids.util.PlanChecker;
import org.apache.doris.nereids.util.PlanConstructor;
import com.google.common.collect.ImmutableList;
import org.junit.jupiter.api.Test;
class PushdownMinMaxThroughJoinTest implements MemoPatternMatchSupported {
private static final LogicalOlapScan scan1 = PlanConstructor.newLogicalOlapScan(0, "t1", 0);
private static final LogicalOlapScan scan2 = PlanConstructor.newLogicalOlapScan(1, "t2", 0);
private static final LogicalOlapScan scan3 = PlanConstructor.newLogicalOlapScan(2, "t3", 0);
private static final LogicalOlapScan scan4 = PlanConstructor.newLogicalOlapScan(3, "t4", 0);
@Test
void testSingleJoin() {
Alias min = new Min(scan1.getOutput().get(0)).alias("min");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(scan1.getOutput().get(0), min))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.applyTopDown(new PushdownMinMaxThroughJoin())
.printlnTree();
}
@Test
void testMultiJoin() {
Alias min = new Min(scan1.getOutput().get(0)).alias("min");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.join(scan3, JoinType.INNER_JOIN, Pair.of(0, 0))
.join(scan4, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(scan1.getOutput().get(0), min))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.applyTopDown(new PushdownMinMaxThroughJoin())
.printlnTree();
}
@Test
void testAggNotOutputGroupBy() {
// agg don't output group by
Alias min = new Min(scan1.getOutput().get(0)).alias("min");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.join(scan3, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(min))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.applyTopDown(new PushdownMinMaxThroughJoin())
.printlnTree();
}
@Test
void testBothSideSingleJoin() {
Alias min = new Min(scan1.getOutput().get(1)).alias("min");
Alias max = new Max(scan2.getOutput().get(1)).alias("max");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(scan1.getOutput().get(0), min, max))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.printlnTree()
.applyTopDown(new PushdownMinMaxThroughJoin())
.printlnTree();
}
@Test
void testBothSide() {
Alias min = new Min(scan1.getOutput().get(1)).alias("min");
Alias max = new Max(scan3.getOutput().get(1)).alias("max");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.join(scan3, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(min, max))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.applyTopDown(new PushdownMinMaxThroughJoin())
.printlnTree();
}
}

View File

@ -0,0 +1,104 @@
// 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.common.Pair;
import org.apache.doris.nereids.trees.expressions.Alias;
import org.apache.doris.nereids.trees.expressions.functions.agg.Sum;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
import org.apache.doris.nereids.util.LogicalPlanBuilder;
import org.apache.doris.nereids.util.MemoPatternMatchSupported;
import org.apache.doris.nereids.util.MemoTestUtils;
import org.apache.doris.nereids.util.PlanChecker;
import org.apache.doris.nereids.util.PlanConstructor;
import com.google.common.collect.ImmutableList;
import org.junit.jupiter.api.Test;
class PushdownSumThroughJoinTest implements MemoPatternMatchSupported {
private static final LogicalOlapScan scan1 = PlanConstructor.newLogicalOlapScan(0, "t1", 0);
private static final LogicalOlapScan scan2 = PlanConstructor.newLogicalOlapScan(1, "t2", 0);
private static final LogicalOlapScan scan3 = PlanConstructor.newLogicalOlapScan(2, "t3", 0);
private static final LogicalOlapScan scan4 = PlanConstructor.newLogicalOlapScan(3, "t4", 0);
@Test
void testSingleJoinLeftSum() {
Alias sum = new Sum(scan1.getOutput().get(1)).alias("sum");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(scan1.getOutput().get(0), sum))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.printlnTree()
.applyTopDown(new PushdownSumThroughJoin())
.matches(
logicalAggregate(
logicalJoin(
logicalAggregate(),
logicalAggregate()
)
)
);
}
@Test
void testSingleJoinRightSum() {
Alias sum = new Sum(scan2.getOutput().get(1)).alias("sum");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(scan1.getOutput().get(0), sum))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.printlnTree()
.applyTopDown(new PushdownSumThroughJoin())
.matches(
logicalAggregate(
logicalJoin(
logicalAggregate(),
logicalAggregate()
)
)
);
}
@Test
void testAggNotOutputGroupBy() {
// agg don't output group by
Alias sum = new Sum(scan1.getOutput().get(1)).alias("sum");
LogicalPlan plan = new LogicalPlanBuilder(scan1)
.join(scan2, JoinType.INNER_JOIN, Pair.of(0, 0))
.aggGroupUsingIndex(ImmutableList.of(0), ImmutableList.of(sum))
.build();
PlanChecker.from(MemoTestUtils.createConnectContext(), plan)
.printlnTree()
.applyTopDown(new PushdownSumThroughJoin())
.matches(
logicalAggregate(
logicalJoin(
logicalAggregate(),
logicalAggregate()
)
)
);
}
}

View File

@ -346,7 +346,12 @@ public class PlanChecker {
}
private PlanChecker applyExploration(Group group, Rule rule) {
// copy children expression, because group may be changed after apply rule.
List<GroupExpression> logicalExpressions = Lists.newArrayList(group.getLogicalExpressions());
// due to mergeGroup, the children Group of groupExpression may be replaced, so we need to use lambda to
// get the child to make we can get child at the time we use child.
// If we use for child: groupExpression.children(), it means that we take it in advance. It may cause NPE,
// work flow: get children() to get left, right -> copyIn left() -> mergeGroup -> right is merged -> NPE
for (int i = 0; i < logicalExpressions.size(); i++) {
final int childIdx = i;
applyExploration(() -> logicalExpressions.get(childIdx), rule);