[Improvement](Nereids) Support aggregate rewrite by materialized view with complex expression (#30440)

materialized view definition is

>            select
>            sum(o_totalprice) as sum_total,
>            max(o_totalprice) as max_total,
>            min(o_totalprice) as min_total,
>           count(*) as count_all,
>            bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) >cnt_1,
>            bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as >cnt_2
>            from lineitem
>            left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate;
   

the query following can be rewritten by materialized view above.
it use the aggregate fuction arithmetic calculation in the select 

>            select
>            count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2,
>            (sum(o_totalprice) + min(o_totalprice)) * count(*),
>            min(o_totalprice) + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null >end)
>            from lineitem
>            left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate;
This commit is contained in:
seawinde
2024-01-29 16:47:26 +08:00
committed by yiguolei
parent edeec320d3
commit dce6c8bd65
5 changed files with 402 additions and 28 deletions

View File

@ -38,6 +38,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.BitmapUnionCount
import org.apache.doris.nereids.trees.expressions.functions.agg.CouldRollUp;
import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
import org.apache.doris.nereids.trees.expressions.functions.scalar.ToBitmap;
import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter;
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.LogicalProject;
@ -65,6 +66,8 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
protected static final Multimap<Function, Expression>
AGGREGATE_ROLL_UP_EQUIVALENT_FUNCTION_MAP = ArrayListMultimap.create();
protected static final AggregateExpressionRewriter AGGREGATE_EXPRESSION_REWRITER =
new AggregateExpressionRewriter();
static {
// support count distinct roll up
@ -156,7 +159,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
= topPlanSplitToGroupAndFunction(queryTopPlanAndAggPair);
Set<? extends Expression> queryTopPlanFunctionSet = queryGroupAndFunctionPair.value();
// try to rewrite, contains both roll up aggregate functions and aggregate group expression
List<NamedExpression> finalAggregateExpressions = new ArrayList<>();
List<NamedExpression> finalOutputExpressions = new ArrayList<>();
List<Expression> finalGroupExpressions = new ArrayList<>();
List<? extends Expression> queryExpressions = queryTopPlan.getExpressions();
// permute the mv expr mapping to query based
@ -169,32 +172,29 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
Expression queryFunctionShuttled = ExpressionUtils.shuttleExpressionWithLineage(
topExpression,
queryTopPlan);
// try to roll up
List<Object> queryFunctions =
queryFunctionShuttled.collectFirst(expr -> expr instanceof AggregateFunction);
if (queryFunctions.isEmpty()) {
materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(),
Pair.of("Can not found query function",
String.format("queryFunctionShuttled = %s", queryFunctionShuttled)));
return null;
}
Function rollupAggregateFunction = rollup((AggregateFunction) queryFunctions.get(0),
queryFunctionShuttled, mvExprToMvScanExprQueryBased);
if (rollupAggregateFunction == null) {
AggregateExpressionRewriteContext context = new AggregateExpressionRewriteContext(
false, mvExprToMvScanExprQueryBased, queryTopPlan);
// queryFunctionShuttled maybe sum(column) + count(*), so need to use expression rewriter
Expression rollupedExpression = queryFunctionShuttled.accept(AGGREGATE_EXPRESSION_REWRITER,
context);
if (!context.isValid()) {
materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(),
Pair.of("Query function roll up fail",
String.format("queryFunction = %s,\n queryFunctionShuttled = %s,\n"
+ "mvExprToMvScanExprQueryBased = %s",
queryFunctions.get(0), queryFunctionShuttled,
mvExprToMvScanExprQueryBased)));
String.format("queryFunctionShuttled = %s,\n mvExprToMvScanExprQueryBased = %s",
queryFunctionShuttled, mvExprToMvScanExprQueryBased)));
return null;
}
finalAggregateExpressions.add(new Alias(rollupAggregateFunction));
finalOutputExpressions.add(new Alias(rollupedExpression));
} else {
// if group by expression, try to rewrite group by expression
Expression queryGroupShuttledExpr =
ExpressionUtils.shuttleExpressionWithLineage(topExpression, queryTopPlan);
if (!mvExprToMvScanExprQueryBased.containsKey(queryGroupShuttledExpr)) {
AggregateExpressionRewriteContext context = new AggregateExpressionRewriteContext(
true, mvExprToMvScanExprQueryBased, queryTopPlan);
// group by expression maybe group by a + b, so we need expression rewriter
Expression rewrittenGroupByExpression = queryGroupShuttledExpr.accept(AGGREGATE_EXPRESSION_REWRITER,
context);
if (!context.isValid()) {
// group expr can not rewrite by view
materializationContext.recordFailReason(queryStructInfo.getOriginalPlanId(),
Pair.of("View dimensions doesn't not cover the query dimensions",
@ -202,9 +202,10 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
mvExprToMvScanExprQueryBased, queryGroupShuttledExpr)));
return null;
}
Expression expression = mvExprToMvScanExprQueryBased.get(queryGroupShuttledExpr);
finalAggregateExpressions.add((NamedExpression) expression);
finalGroupExpressions.add(expression);
NamedExpression groupByExpression = rewrittenGroupByExpression instanceof NamedExpression
? (NamedExpression) rewrittenGroupByExpression : new Alias(rewrittenGroupByExpression);
finalOutputExpressions.add(groupByExpression);
finalGroupExpressions.add(groupByExpression);
}
}
// add project to guarantee group by column ref is slot reference,
@ -229,7 +230,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
return (NamedExpression) expr;
})
.collect(Collectors.toList());
finalAggregateExpressions = finalAggregateExpressions.stream()
finalOutputExpressions = finalOutputExpressions.stream()
.map(expr -> {
ExprId exprId = expr.getExprId();
if (projectOutPutExprIdMap.containsKey(exprId)) {
@ -238,7 +239,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
return expr;
})
.collect(Collectors.toList());
return new LogicalAggregate(finalGroupExpressions, finalAggregateExpressions, mvProject);
return new LogicalAggregate(finalGroupExpressions, finalOutputExpressions, mvProject);
}
private boolean isGroupByEquals(Pair<Plan, LogicalAggregate<Plan>> queryTopPlanAndAggPair,
@ -273,7 +274,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
* the queryAggregateFunction is max(a), queryAggregateFunctionShuttled is max(a) + 1
* mvExprToMvScanExprQueryBased is { max(a) : MTMVScan(output#0) }
*/
private Function rollup(AggregateFunction queryAggregateFunction,
private static Function rollup(AggregateFunction queryAggregateFunction,
Expression queryAggregateFunctionShuttled,
Map<Expression, Expression> mvExprToMvScanExprQueryBased) {
if (!(queryAggregateFunction instanceof CouldRollUp)) {
@ -310,7 +311,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
// Check the aggregate function can roll up or not, return true if could roll up
// if view aggregate function is distinct or is in the un supported rollup functions, it doesn't support
// roll up.
private boolean canRollup(Expression rollupExpression) {
private static boolean canRollup(Expression rollupExpression) {
if (rollupExpression == null) {
return false;
}
@ -402,7 +403,7 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
* This will check the count(distinct a) in query is equivalent to bitmap_union(to_bitmap(a)) in mv,
* and then check their arguments is equivalent.
*/
private boolean isAggregateFunctionEquivalent(Function queryFunction, Function viewFunction) {
private static boolean isAggregateFunctionEquivalent(Function queryFunction, Function viewFunction) {
if (queryFunction.equals(viewFunction)) {
return true;
}
@ -438,9 +439,109 @@ public abstract class AbstractMaterializedViewAggregateRule extends AbstractMate
* actualFunction is bitmap_union(to_bitmap(case when a = 5 then 1 else 2 end))
* after extracting, the return argument is: case when a = 5 then 1 else 2 end
*/
private List<Expression> extractArguments(Expression functionWithAny, Function actualFunction) {
private static List<Expression> extractArguments(Expression functionWithAny, Function actualFunction) {
Set<Object> exprSetToRemove = functionWithAny.collectToSet(expr -> !(expr instanceof Any));
return actualFunction.collectFirst(expr ->
exprSetToRemove.stream().noneMatch(exprToRemove -> exprToRemove.equals(expr)));
}
/**
* Aggregate expression rewriter which is responsible for rewriting group by and
* aggregate function expression
*/
protected static class AggregateExpressionRewriter
extends DefaultExpressionRewriter<AggregateExpressionRewriteContext> {
@Override
public Expression visitAggregateFunction(AggregateFunction aggregateFunction,
AggregateExpressionRewriteContext rewriteContext) {
if (!rewriteContext.isValid()) {
return aggregateFunction;
}
Expression queryFunctionShuttled = ExpressionUtils.shuttleExpressionWithLineage(
aggregateFunction,
rewriteContext.getQueryTopPlan());
Function rollupAggregateFunction = rollup(aggregateFunction, queryFunctionShuttled,
rewriteContext.getMvExprToMvScanExprQueryBasedMapping());
if (rollupAggregateFunction == null) {
rewriteContext.setValid(false);
return aggregateFunction;
}
return rollupAggregateFunction;
}
@Override
public Expression visitSlot(Slot slot, AggregateExpressionRewriteContext rewriteContext) {
if (!rewriteContext.isValid()) {
return slot;
}
if (rewriteContext.getMvExprToMvScanExprQueryBasedMapping().containsKey(slot)) {
return rewriteContext.getMvExprToMvScanExprQueryBasedMapping().get(slot);
}
rewriteContext.setValid(false);
return slot;
}
@Override
public Expression visit(Expression expr, AggregateExpressionRewriteContext rewriteContext) {
if (!rewriteContext.isValid()) {
return expr;
}
// for group by expression try to get corresponding expression directly
if (rewriteContext.isOnlyContainGroupByExpression()
&& rewriteContext.getMvExprToMvScanExprQueryBasedMapping().containsKey(expr)) {
return rewriteContext.getMvExprToMvScanExprQueryBasedMapping().get(expr);
}
List<Expression> newChildren = new ArrayList<>(expr.arity());
boolean hasNewChildren = false;
for (Expression child : expr.children()) {
Expression newChild = child.accept(this, rewriteContext);
if (!rewriteContext.isValid()) {
return expr;
}
if (newChild != child) {
hasNewChildren = true;
}
newChildren.add(newChild);
}
return hasNewChildren ? expr.withChildren(newChildren) : expr;
}
}
/**
* AggregateExpressionRewriteContext
*/
protected static class AggregateExpressionRewriteContext {
private boolean valid = true;
private final boolean onlyContainGroupByExpression;
private final Map<Expression, Expression> mvExprToMvScanExprQueryBasedMapping;
private final Plan queryTopPlan;
public AggregateExpressionRewriteContext(boolean onlyContainGroupByExpression,
Map<Expression, Expression> mvExprToMvScanExprQueryBasedMapping, Plan queryTopPlan) {
this.onlyContainGroupByExpression = onlyContainGroupByExpression;
this.mvExprToMvScanExprQueryBasedMapping = mvExprToMvScanExprQueryBasedMapping;
this.queryTopPlan = queryTopPlan;
}
public boolean isValid() {
return valid;
}
public void setValid(boolean valid) {
this.valid = valid;
}
public boolean isOnlyContainGroupByExpression() {
return onlyContainGroupByExpression;
}
public Map<Expression, Expression> getMvExprToMvScanExprQueryBasedMapping() {
return mvExprToMvScanExprQueryBasedMapping;
}
public Plan getQueryTopPlan() {
return queryTopPlan;
}
}
}

View File

@ -193,6 +193,30 @@
2023-12-11 3 43.20 43.20 43.20 1 \N \N 0 1 1
2023-12-12 3 57.40 56.20 1.20 2 \N \N 0 1 1
-- !query25_3_before --
2023-12-08 5 21.00 10.50 9.50 2 \N \N 1 0 1 0
2023-12-09 7 11.50 11.50 11.50 1 \N \N 1 0 1 0
2023-12-10 6 67.00 33.50 12.50 2 \N \N 1 0 1 0
2023-12-11 6 43.20 43.20 43.20 1 \N \N 0 1 1 1
2023-12-12 5 112.40 56.20 1.20 2 \N \N 0 1 1 1
-- !query25_3_after --
2023-12-08 5 21.00 10.50 9.50 2 \N \N 1 0 1 0
2023-12-09 7 11.50 11.50 11.50 1 \N \N 1 0 1 0
2023-12-10 6 67.00 33.50 12.50 2 \N \N 1 0 1 0
2023-12-11 6 43.20 43.20 43.20 1 \N \N 0 1 1 1
2023-12-12 5 112.40 56.20 1.20 2 \N \N 0 1 1 1
-- !query25_4_before --
2 3 2023-12-08 20.00 23.00
2 3 2023-12-12 57.40 60.40
2 4 2023-12-10 46.00 50.00
-- !query25_4_after --
2 3 2023-12-08 20.00 23.00
2 3 2023-12-12 57.40 60.40
2 4 2023-12-10 46.00 50.00
-- !query1_1_before --
1 yy 0 0 11.50 11.50 11.50 1
@ -261,6 +285,12 @@
-- !query29_1_after --
0 178.10 1.20 8
-- !query29_2_before --
0 1434.40 1.20
-- !query29_2_after --
0 1434.40 1.20
-- !query30_0_before --
4 4 68 100.0000 36.5000
6 1 0 22.0000 57.2000

View File

@ -85,6 +85,18 @@
2 4 2023-12-10 46.00 33.50 12.50 2 0
3 3 2023-12-11 43.20 43.20 43.20 1 0
-- !query15_1_before --
2 20231211 30.50 20.00 10.50 9.50 2 0 2
2 20231214 79.50 46.00 33.50 12.50 2 0 2
2 20231215 113.60 57.40 56.20 1.20 2 0 2
3 20231214 86.40 43.20 43.20 43.20 1 0 1
-- !query15_0_after --
2 20231211 30.50 20.00 10.50 9.50 2 0 2
2 20231214 79.50 46.00 33.50 12.50 2 0 2
2 20231215 113.60 57.40 56.20 1.20 2 0 2
3 20231214 86.40 43.20 43.20 43.20 1 0 1
-- !query16_0_before --
2 3 2023-12-08 20.00 10.50 9.50 2 0
2 3 2023-12-12 57.40 56.20 1.20 2 0
@ -99,6 +111,20 @@
3 3 2023-12-11 43.20 43.20 43.20 1 0
4 3 2023-12-09 11.50 11.50 11.50 1 0
-- !query16_1_before --
3 2023-12-08 20.00 10.50 9.50 2 0
3 2023-12-09 11.50 11.50 11.50 1 0
3 2023-12-11 43.20 43.20 43.20 1 0
3 2023-12-12 57.40 56.20 1.20 2 0
4 2023-12-10 46.00 33.50 12.50 2 0
-- !query16_1_after --
3 2023-12-08 20.00 10.50 9.50 2 0
3 2023-12-09 11.50 11.50 11.50 1 0
3 2023-12-11 43.20 43.20 43.20 1 0
3 2023-12-12 57.40 56.20 1.20 2 0
4 2023-12-10 46.00 33.50 12.50 2 0
-- !query17_0_before --
3 3 2023-12-11 43.20 43.20 43.20 1 0
@ -177,6 +203,20 @@
2023-12-11 3 3 3 43.20 43.20 43.20 1
2023-12-12 2 3 3 57.40 56.20 1.20 2
-- !query19_3_before --
2 23.00 2023-12-08 20.00 10.50 9.50 29.50 2
2 50.00 2023-12-10 46.00 33.50 12.50 58.50 2
2 60.40 2023-12-12 57.40 56.20 1.20 58.60 2
3 46.20 2023-12-11 43.20 43.20 43.20 86.40 1
4 14.50 2023-12-09 11.50 11.50 11.50 23.00 1
-- !query19_3_after --
2 23.00 2023-12-08 20.00 10.50 9.50 29.50 2
2 50.00 2023-12-10 46.00 33.50 12.50 58.50 2
2 60.40 2023-12-12 57.40 56.20 1.20 58.60 2
3 46.20 2023-12-11 43.20 43.20 43.20 86.40 1
4 14.50 2023-12-09 11.50 11.50 11.50 23.00 1
-- !query20_0_before --
0 0 0 0 0 0 0 0 0 0 0 0

View File

@ -886,6 +886,75 @@ suite("aggregate_with_roll_up") {
sql """ DROP MATERIALIZED VIEW IF EXISTS mv25_2"""
// bitmap_union roll up to bitmap_union_count
def mv25_3 = """
select l_shipdate, o_orderdate, l_partkey, l_suppkey,
sum(o_totalprice) as sum_total,
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
count(*) as count_all,
bitmap_union(to_bitmap(case when o_shippriority > 0 and o_orderkey IN (1, 2, 3) then o_custkey else null end)) cnt_1,
bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (3, 4, 5) then o_custkey else null end)) as cnt_2
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate
group by
l_shipdate,
o_orderdate,
l_partkey,
l_suppkey;
"""
def query25_3 = """
select o_orderdate, l_suppkey + l_partkey,
sum(o_totalprice) + max(o_totalprice) - min(o_totalprice),
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
count(*) as count_all,
bitmap_union(to_bitmap(case when o_shippriority > 0 and o_orderkey IN (1, 2, 3) then o_custkey else null end)),
bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (3, 4, 5) then o_custkey else null end)),
bitmap_union_count(to_bitmap(case when o_shippriority > 0 and o_orderkey IN (1, 2, 3) then o_custkey else null end)),
bitmap_union_count(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (3, 4, 5) then o_custkey else null end)),
bitmap_union_count(to_bitmap(case when o_shippriority > 0 and o_orderkey IN (1, 2, 3) then o_custkey else null end)) + bitmap_union_count(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (3, 4, 5) then o_custkey else null end)),
count(distinct case when o_shippriority > 1 and o_orderkey IN (3, 4, 5) then o_custkey else null end)
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate
group by
o_orderdate,
l_suppkey + l_partkey;
"""
order_qt_query25_3_before "${query25_3}"
check_rewrite(mv25_3, query25_3, "mv25_3")
order_qt_query25_3_after "${query25_3}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv25_3"""
def mv25_4 = """
select l_shipdate, o_orderdate, l_partkey, l_suppkey, sum(o_totalprice) as sum_total,
sum(o_totalprice) + l_suppkey
from lineitem
left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate
group by
l_shipdate,
o_orderdate,
l_partkey,
l_suppkey;
"""
def query25_4 = """
select t1.l_partkey, t1.l_suppkey, o_orderdate, sum(o_totalprice), sum(o_totalprice) + t1.l_suppkey
from (select * from lineitem where l_partkey = 2 ) t1
left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey,
l_suppkey;
"""
order_qt_query25_4_before "${query25_4}"
check_rewrite(mv25_4, query25_4, "mv25_4")
order_qt_query25_4_after "${query25_4}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv25_4"""
// single table
// filter + use roll up dimension
def mv1_1 = """
@ -1122,6 +1191,33 @@ suite("aggregate_with_roll_up") {
order_qt_query29_1_after "${query29_1}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_1"""
// mv and query both are scalar aggregate, and query calc the aggregate function
def mv29_2 = """
select
sum(o_totalprice) as sum_total,
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
count(*) as count_all,
bitmap_union(to_bitmap(case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)) cnt_1,
bitmap_union(to_bitmap(case when o_shippriority > 2 and o_orderkey IN (2) then o_custkey else null end)) as cnt_2
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate;
"""
def query29_2 = """
select
count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end) as cnt_2,
(sum(o_totalprice) + min(o_totalprice)) * count(*),
min(o_totalprice) + count(distinct case when O_SHIPPRIORITY > 2 and o_orderkey IN (2) then o_custkey else null end)
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate;
"""
order_qt_query29_2_before "${query29_2}"
check_rewrite(mv29_2, query29_2, "mv29_2")
order_qt_query29_2_after "${query29_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv29_2"""
// join input has simple agg, simple agg which can not contains rollup, cube
// can not rewrite, because avg doesn't support roll up now
def mv30_0 = """

View File

@ -502,6 +502,43 @@ suite("aggregate_without_roll_up") {
order_qt_query15_0_after "${query15_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_0"""
def mv15_1 = """
select o_orderdate, l_partkey, l_suppkey,
sum(o_totalprice) as sum_total,
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
count(*) as count_all,
count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count
from lineitem
left join (select * from orders where o_orderstatus = 'o') t2
on lineitem.l_orderkey = o_orderkey and l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey,
l_suppkey;
"""
def query15_1 = """
select t1.l_partkey, t1.l_suppkey + o_orderdate,
sum(o_totalprice) + max(o_totalprice),
sum(o_totalprice),
max(o_totalprice),
min(o_totalprice),
count(*),
count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end),
count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) + count(*)
from (select * from lineitem where l_partkey in (2, 3)) t1
left join (select * from orders where o_orderstatus = 'o') t2
on t1.l_orderkey = o_orderkey and t1.l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey,
l_suppkey;
"""
order_qt_query15_1_before "${query15_1}"
check_rewrite(mv15_1, query15_1, "mv15_1")
order_qt_query15_0_after "${query15_1}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv15_1"""
// filter outside + left
def mv16_0 = "select o_orderdate, l_partkey, l_suppkey, " +
"sum(o_totalprice) as sum_total, " +
@ -533,6 +570,39 @@ suite("aggregate_without_roll_up") {
order_qt_query16_0_after "${query16_0}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_0"""
// should not rewrite, because query has the dimension which is not in view
def mv16_1 = """
select o_orderdate, l_partkey,
sum(o_totalprice) as sum_total,
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
count(*) as count_all,
count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end) as distinct_count
from lineitem
left join orders on lineitem.l_orderkey = orders.o_orderkey and l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey;
"""
def query16_1 = """
select t1.l_suppkey, o_orderdate,
sum(o_totalprice),
max(o_totalprice),
min(o_totalprice),
count(*),
count(distinct case when o_shippriority > 1 and o_orderkey IN (1, 3) then o_custkey else null end)
from lineitem t1
left join orders on t1.l_orderkey = orders.o_orderkey and t1.l_shipdate = o_orderdate
where l_partkey in (1, 2 ,3, 4)
group by
o_orderdate,
l_suppkey;
"""
order_qt_query16_1_before "${query16_1}"
check_not_match(mv16_1, query16_1, "mv16_1")
order_qt_query16_1_after "${query16_1}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv16_1"""
// filter outside + right
def mv17_0 = "select o_orderdate, l_partkey, l_suppkey, " +
@ -772,6 +842,43 @@ suite("aggregate_without_roll_up") {
order_qt_query19_2_after "${query19_2}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_2"""
// aggregate function and group by expression is complex
def mv19_3 = """
select o_orderdate, l_partkey, l_suppkey + sum(o_totalprice),
sum(o_totalprice),
max(o_totalprice) as max_total,
min(o_totalprice) as min_total,
min(o_totalprice) + sum(o_totalprice),
count(*) as count_all
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey,
l_suppkey;
"""
def query19_3 = """
select l_partkey, l_suppkey + sum(o_totalprice), o_orderdate,
sum(o_totalprice),
max(o_totalprice),
min(o_totalprice),
min(o_totalprice) + sum(o_totalprice),
count(*)
from lineitem
left join orders on l_orderkey = o_orderkey and l_shipdate = o_orderdate
group by
o_orderdate,
l_partkey,
l_suppkey;
"""
order_qt_query19_3_before "${query19_3}"
check_rewrite(mv19_3, query19_3, "mv19_3")
order_qt_query19_3_after "${query19_3}"
sql """ DROP MATERIALIZED VIEW IF EXISTS mv19_0"""
// without group, scalar aggregate
def mv20_0 = "select count(distinct case when O_SHIPPRIORITY > 1 and O_ORDERKEY IN (1, 3) then O_ORDERSTATUS else null end) as filter_cnt_1, " +
"count(distinct case when O_SHIPPRIORITY > 2 and O_ORDERKEY IN (2) then O_ORDERSTATUS else null end) as filter_cnt_2, " +