[feature](Nereids) support statement having aggregate function in order by list (#13976)

1. add a feature that support statement having aggregate function in order by list. such as:
    SELECT COUNT(*) FROM t GROUP BY c1 ORDER BY COUNT(*) DESC;
2. add clickbench analyze unit tests
This commit is contained in:
morrySnow
2022-11-07 17:01:31 +08:00
committed by GitHub
parent 0031304015
commit 22b4c6af20
10 changed files with 779 additions and 82 deletions

View File

@ -23,7 +23,7 @@ import org.apache.doris.nereids.rules.analysis.BindRelation;
import org.apache.doris.nereids.rules.analysis.BindSlotReference;
import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate;
import org.apache.doris.nereids.rules.analysis.RegisterCTE;
import org.apache.doris.nereids.rules.analysis.ResolveHaving;
import org.apache.doris.nereids.rules.analysis.ResolveAggregateFunctions;
import org.apache.doris.nereids.rules.analysis.Scope;
import org.apache.doris.nereids.rules.analysis.UserAuthentication;
@ -52,8 +52,11 @@ public class AnalyzeRulesJob extends BatchRulesJob {
new UserAuthentication(),
new BindSlotReference(scope),
new BindFunction(),
new ResolveHaving(),
new ProjectToGlobalAggregate())
)));
new ProjectToGlobalAggregate()
)),
topDownBatch(ImmutableList.of(
new ResolveAggregateFunctions()
))
));
}
}

View File

@ -129,6 +129,7 @@ import org.apache.doris.nereids.trees.expressions.literal.SmallIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral;
import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral;
import org.apache.doris.nereids.trees.plans.JoinType;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.commands.Command;
import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel;
@ -160,7 +161,6 @@ import org.antlr.v4.runtime.tree.TerminalNode;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
@ -171,10 +171,10 @@ import java.util.stream.Collectors;
/**
* Build a logical plan tree with unbounded nodes.
*/
@SuppressWarnings({"OptionalUsedAsFieldOrParameterType", "OptionalGetWithoutIsPresent"})
public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
private StatementContext statementContext = new StatementContext();
@SuppressWarnings("unchecked")
protected <T> T typedVisit(ParseTree ctx) {
return (T) ctx.accept(this);
}
@ -211,7 +211,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
public List<Pair<LogicalPlan, StatementContext>> visitMultiStatements(MultiStatementsContext ctx) {
List<Pair<LogicalPlan, StatementContext>> logicalPlans = Lists.newArrayList();
for (org.apache.doris.nereids.DorisParser.StatementContext statement : ctx.statement()) {
statementContext = new StatementContext();
StatementContext statementContext = new StatementContext();
if (ConnectContext.get() != null) {
ConnectContext.get().setStatementContext(statementContext);
}
@ -233,19 +233,19 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
}
/**
* processs CTE's alias queries and column aliases
* process CTE's alias queries and column aliases
*/
@Override
public LogicalSubQueryAlias visitAliasQuery(AliasQueryContext ctx) {
public LogicalSubQueryAlias<Plan> visitAliasQuery(AliasQueryContext ctx) {
return ParserUtils.withOrigin(ctx, () -> {
LogicalPlan queryPlan = plan(ctx.query());
List<String> columnNames = null;
if (ctx.columnAliases() != null) {
columnNames = ctx.columnAliases().identifier().stream()
.map(id -> id.getText())
.map(RuleContext::getText)
.collect(ImmutableList.toImmutableList());
}
return new LogicalSubQueryAlias(ctx.identifier().getText(), Optional.ofNullable(columnNames), queryPlan);
return new LogicalSubQueryAlias<>(ctx.identifier().getText(), Optional.ofNullable(columnNames), queryPlan);
});
}
@ -556,7 +556,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
public UnboundFunction visitExtract(DorisParser.ExtractContext ctx) {
return ParserUtils.withOrigin(ctx, () -> {
String functionName = ctx.field.getText();
return new UnboundFunction(functionName, false, false, Arrays.asList(getExpression(ctx.source)));
return new UnboundFunction(functionName, false, false,
Collections.singletonList(getExpression(ctx.source)));
});
}

View File

@ -33,9 +33,10 @@ public enum RuleType {
BINDING_ONE_ROW_RELATION_SLOT(RuleTypeClass.REWRITE),
BINDING_RELATION(RuleTypeClass.REWRITE),
BINDING_PROJECT_SLOT(RuleTypeClass.REWRITE),
BINDING_FILTER_SLOT(RuleTypeClass.REWRITE),
BINDING_JOIN_SLOT(RuleTypeClass.REWRITE),
BINDING_FILTER_SLOT(RuleTypeClass.REWRITE),
BINDING_AGGREGATE_SLOT(RuleTypeClass.REWRITE),
BINDING_HAVING_SLOT(RuleTypeClass.REWRITE),
BINDING_SORT_SLOT(RuleTypeClass.REWRITE),
BINDING_LIMIT_SLOT(RuleTypeClass.REWRITE),
BINDING_ONE_ROW_RELATION_FUNCTION(RuleTypeClass.REWRITE),
@ -43,9 +44,11 @@ public enum RuleType {
BINDING_AGGREGATE_FUNCTION(RuleTypeClass.REWRITE),
BINDING_SUBQUERY_ALIAS_SLOT(RuleTypeClass.REWRITE),
BINDING_FILTER_FUNCTION(RuleTypeClass.REWRITE),
BINDING_HAVING_SLOT(RuleTypeClass.REWRITE),
BINDING_HAVING_FUNCTION(RuleTypeClass.REWRITE),
RESOLVE_HAVING(RuleTypeClass.REWRITE),
BINDING_SORT_FUNCTION(RuleTypeClass.REWRITE),
RESOLVE_HAVING_AGGREGATE_FUNCTIONS(RuleTypeClass.REWRITE),
RESOLVE_SORT_AGGREGATE_FUNCTIONS(RuleTypeClass.REWRITE),
RESOLVE_SORT_HAVING_AGGREGATE_FUNCTIONS(RuleTypeClass.REWRITE),
RESOLVE_PROJECT_ALIAS(RuleTypeClass.REWRITE),
RESOLVE_AGGREGATE_ALIAS(RuleTypeClass.REWRITE),

View File

@ -21,6 +21,7 @@ import org.apache.doris.analysis.ArithmeticExpr.Operator;
import org.apache.doris.catalog.Env;
import org.apache.doris.catalog.FunctionRegistry;
import org.apache.doris.nereids.analyzer.UnboundFunction;
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;
@ -37,6 +38,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalHaving;
import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
import com.google.common.collect.ImmutableList;
@ -91,11 +93,24 @@ public class BindFunction implements AnalysisRuleFactory {
})
),
RuleType.BINDING_HAVING_FUNCTION.build(
logicalHaving(logicalAggregate()).thenApply(ctx -> {
LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root;
logicalHaving().thenApply(ctx -> {
LogicalHaving<GroupPlan> having = ctx.root;
List<Expression> predicates = bind(having.getExpressions(), ctx.connectContext.getEnv());
return new LogicalHaving<>(predicates.get(0), having.child());
})
),
RuleType.BINDING_SORT_FUNCTION.build(
logicalSort().thenApply(ctx -> {
LogicalSort<GroupPlan> sort = ctx.root;
List<OrderKey> orderKeys = sort.getOrderKeys().stream()
.map(orderKey -> new OrderKey(
FunctionBinder.INSTANCE.bind(orderKey.getExpr(), ctx.connectContext.getEnv()),
orderKey.isAsc(),
orderKey.isNullFirst()
))
.collect(ImmutableList.toImmutableList());
return new LogicalSort<>(orderKeys, sort.child());
})
)
);
}

View File

@ -129,6 +129,20 @@ public class BindSlotReference implements AnalysisRuleFactory {
return agg.withGroupByAndOutput(groupBy, output);
})
),
RuleType.BINDING_SORT_SLOT.build(
logicalSort(logicalAggregate()).when(Plan::canBind).thenApply(ctx -> {
LogicalSort<LogicalAggregate<GroupPlan>> sort = ctx.root;
LogicalAggregate<GroupPlan> aggregate = sort.child();
return bindSortWithAggregateFunction(sort, aggregate, ctx.cascadesContext);
})
),
RuleType.BINDING_SORT_SLOT.build(
logicalSort(logicalHaving(logicalAggregate())).when(Plan::canBind).thenApply(ctx -> {
LogicalSort<LogicalHaving<LogicalAggregate<GroupPlan>>> sort = ctx.root;
LogicalAggregate<GroupPlan> aggregate = sort.child().child();
return bindSortWithAggregateFunction(sort, aggregate, ctx.cascadesContext);
})
),
RuleType.BINDING_SORT_SLOT.build(
logicalSort().when(Plan::canBind).thenApply(ctx -> {
LogicalSort<GroupPlan> sort = ctx.root;
@ -143,7 +157,7 @@ public class BindSlotReference implements AnalysisRuleFactory {
})
),
RuleType.BINDING_HAVING_SLOT.build(
logicalHaving(logicalAggregate()).thenApply(ctx -> {
logicalHaving(logicalAggregate()).when(Plan::canBind).thenApply(ctx -> {
LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root;
LogicalAggregate<GroupPlan> aggregate = having.child();
// We should deduplicate the slots, otherwise the binding process will fail due to the
@ -177,6 +191,23 @@ public class BindSlotReference implements AnalysisRuleFactory {
);
}
private Plan bindSortWithAggregateFunction(
LogicalSort<? extends Plan> sort, LogicalAggregate<? extends Plan> aggregate, CascadesContext ctx) {
// We should deduplicate the slots, otherwise the binding process will fail due to the
// ambiguous slots exist.
Set<Slot> boundSlots = Stream.concat(Stream.of(aggregate), aggregate.children().stream())
.flatMap(plan -> plan.getOutput().stream())
.collect(Collectors.toSet());
List<OrderKey> sortItemList = sort.getOrderKeys()
.stream()
.map(orderKey -> {
Expression item = new SlotBinder(toScope(new ArrayList<>(boundSlots)), sort, ctx)
.bind(orderKey.getExpr());
return new OrderKey(item, orderKey.isAsc(), orderKey.isNullFirst());
}).collect(Collectors.toList());
return new LogicalSort<>(sortItemList, sort.child());
}
private List<NamedExpression> flatBoundStar(List<NamedExpression> boundSlots) {
return boundSlots
.stream()

View File

@ -19,6 +19,7 @@ package org.apache.doris.nereids.rules.analysis;
import org.apache.doris.common.Pair;
import org.apache.doris.nereids.exceptions.AnalysisException;
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.Alias;
@ -31,10 +32,11 @@ import org.apache.doris.nereids.trees.plans.GroupPlan;
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.LogicalFilter;
import org.apache.doris.nereids.trees.plans.logical.LogicalHaving;
import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
import org.apache.doris.nereids.trees.plans.logical.LogicalSort;
import org.apache.doris.nereids.util.ExpressionUtils;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Streams;
@ -47,17 +49,60 @@ import java.util.stream.Collectors;
/**
* Resolve having clause to the aggregation.
*/
public class ResolveHaving extends OneAnalysisRuleFactory {
public class ResolveAggregateFunctions implements AnalysisRuleFactory {
@Override
public Rule build() {
return RuleType.RESOLVE_HAVING.build(
logicalHaving(logicalAggregate()).thenApply(ctx -> {
LogicalHaving<LogicalAggregate<GroupPlan>> having = ctx.root;
LogicalAggregate<GroupPlan> aggregate = having.child();
Resolver resolver = new Resolver(aggregate);
resolver.resolve(having.getPredicates());
return createPlan(having, resolver);
})
public List<Rule> buildRules() {
return ImmutableList.of(
RuleType.RESOLVE_SORT_AGGREGATE_FUNCTIONS.build(
logicalSort(logicalAggregate())
.when(sort -> sort.getExpressions().stream()
.anyMatch(e -> e.containsType(AggregateFunction.class)))
.then(sort -> {
LogicalAggregate<GroupPlan> aggregate = sort.child();
Resolver resolver = new Resolver(aggregate);
sort.getExpressions().forEach(resolver::resolve);
return createPlan(resolver, sort.child(), (r, a) -> {
List<OrderKey> newOrderKeys = sort.getOrderKeys().stream()
.map(ok -> new OrderKey(
ExpressionUtils.replace(ok.getExpr(), r.getSubstitution()),
ok.isAsc(),
ok.isNullFirst()))
.collect(ImmutableList.toImmutableList());
return new LogicalSort<>(newOrderKeys, a);
});
})
),
RuleType.RESOLVE_SORT_HAVING_AGGREGATE_FUNCTIONS.build(
logicalSort(logicalHaving(logicalAggregate()))
.when(sort -> sort.getExpressions().stream()
.anyMatch(e -> e.containsType(AggregateFunction.class)))
.then(sort -> {
LogicalAggregate<GroupPlan> aggregate = sort.child().child();
Resolver resolver = new Resolver(aggregate);
sort.getExpressions().forEach(resolver::resolve);
return createPlan(resolver, sort.child().child(), (r, a) -> {
List<OrderKey> newOrderKeys = sort.getOrderKeys().stream()
.map(ok -> new OrderKey(
ExpressionUtils.replace(ok.getExpr(), r.getSubstitution()),
ok.isAsc(),
ok.isNullFirst()))
.collect(ImmutableList.toImmutableList());
return new LogicalSort<>(newOrderKeys, sort.child().withChildren(a));
});
})
),
RuleType.RESOLVE_HAVING_AGGREGATE_FUNCTIONS.build(
logicalHaving(logicalAggregate()).then(having -> {
LogicalAggregate<GroupPlan> aggregate = having.child();
Resolver resolver = new Resolver(aggregate);
resolver.resolve(having.getPredicates());
return createPlan(resolver, having.child(), (r, a) -> {
Expression newPredicates = ExpressionUtils.replace(
having.getPredicates(), r.getSubstitution());
return new LogicalFilter<>(newPredicates, a);
});
})
)
);
}
@ -169,16 +214,20 @@ public class ResolveHaving extends OneAnalysisRuleFactory {
}
}
private Plan createPlan(LogicalHaving<LogicalAggregate<GroupPlan>> having, Resolver resolver) {
LogicalAggregate<GroupPlan> aggregate = having.child();
Expression newPredicates = ExpressionUtils.replace(having.getPredicates(), resolver.getSubstitution());
interface PlanGenerator {
Plan apply(Resolver resolver, LogicalAggregate<Plan> aggregate);
}
private Plan createPlan(Resolver resolver, LogicalAggregate<? extends Plan> aggregate,
PlanGenerator planGenerator) {
List<NamedExpression> projections = aggregate.getOutputExpressions().stream()
.map(NamedExpression::toSlot).collect(Collectors.toList());
List<NamedExpression> newOutputExpressions = Streams.concat(
aggregate.getOutputExpressions().stream(), resolver.getNewOutputSlots().stream()
).collect(Collectors.toList());
LogicalFilter<LogicalAggregate<Plan>> filter = new LogicalFilter<>(newPredicates,
aggregate.withGroupByAndOutput(aggregate.getGroupByExpressions(), newOutputExpressions));
List<NamedExpression> projections = aggregate.getOutputExpressions().stream()
.map(NamedExpression::toSlot).collect(Collectors.toList());
return new LogicalProject<>(projections, filter);
LogicalAggregate<Plan> newAggregate = aggregate.withGroupByAndOutput(
aggregate.getGroupByExpressions(), newOutputExpressions);
Plan plan = planGenerator.apply(resolver, newAggregate);
return new LogicalProject<>(projections, plan);
}
}

View File

@ -0,0 +1,237 @@
// 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.datasets.clickbench;
import org.junit.jupiter.api.Test;
public class AnalyzeClickBenchTest extends ClickBenchTestBase {
@Test
public void q0() {
checkAnalyze(ClickBenchUtils.Q0);
}
@Test
public void q1() {
checkAnalyze(ClickBenchUtils.Q1);
}
@Test
public void q2() {
checkAnalyze(ClickBenchUtils.Q2);
}
@Test
public void q3() {
checkAnalyze(ClickBenchUtils.Q3);
}
@Test
public void q4() {
checkAnalyze(ClickBenchUtils.Q4);
}
@Test
public void q5() {
checkAnalyze(ClickBenchUtils.Q5);
}
@Test
public void q6() {
checkAnalyze(ClickBenchUtils.Q6);
}
@Test
public void q7() {
checkAnalyze(ClickBenchUtils.Q7);
}
@Test
public void q8() {
checkAnalyze(ClickBenchUtils.Q8);
}
@Test
public void q9() {
checkAnalyze(ClickBenchUtils.Q9);
}
@Test
public void q10() {
checkAnalyze(ClickBenchUtils.Q10);
}
@Test
public void q11() {
checkAnalyze(ClickBenchUtils.Q11);
}
@Test
public void q12() {
checkAnalyze(ClickBenchUtils.Q12);
}
@Test
public void q13() {
checkAnalyze(ClickBenchUtils.Q13);
}
@Test
public void q14() {
checkAnalyze(ClickBenchUtils.Q14);
}
@Test
public void q15() {
checkAnalyze(ClickBenchUtils.Q15);
}
@Test
public void q16() {
checkAnalyze(ClickBenchUtils.Q16);
}
@Test
public void q17() {
checkAnalyze(ClickBenchUtils.Q17);
}
// @Test
// public void q18() {
// checkAnalyze(ClickBenchUtils.Q18);
// }
@Test
public void q19() {
checkAnalyze(ClickBenchUtils.Q19);
}
@Test
public void q20() {
checkAnalyze(ClickBenchUtils.Q20);
}
@Test
public void q21() {
checkAnalyze(ClickBenchUtils.Q21);
}
@Test
public void q22() {
checkAnalyze(ClickBenchUtils.Q22);
}
@Test
public void q23() {
checkAnalyze(ClickBenchUtils.Q23);
}
// @Test
// public void q24() {
// checkAnalyze(ClickBenchUtils.Q24);
// }
@Test
public void q25() {
checkAnalyze(ClickBenchUtils.Q25);
}
// @Test
// public void q26() {
// checkAnalyze(ClickBenchUtils.Q26);
// }
@Test
public void q27() {
checkAnalyze(ClickBenchUtils.Q27);
}
// @Test
// public void q28() {
// checkAnalyze(ClickBenchUtils.Q28);
// }
@Test
public void q29() {
checkAnalyze(ClickBenchUtils.Q29);
}
@Test
public void q30() {
checkAnalyze(ClickBenchUtils.Q30);
}
@Test
public void q31() {
checkAnalyze(ClickBenchUtils.Q31);
}
@Test
public void q32() {
checkAnalyze(ClickBenchUtils.Q32);
}
@Test
public void q33() {
checkAnalyze(ClickBenchUtils.Q33);
}
@Test
public void q34() {
checkAnalyze(ClickBenchUtils.Q34);
}
@Test
public void q35() {
checkAnalyze(ClickBenchUtils.Q35);
}
@Test
public void q36() {
checkAnalyze(ClickBenchUtils.Q36);
}
@Test
public void q37() {
checkAnalyze(ClickBenchUtils.Q37);
}
@Test
public void q38() {
checkAnalyze(ClickBenchUtils.Q38);
}
// @Test
// public void q39() {
// checkAnalyze(ClickBenchUtils.Q39);
// }
@Test
public void q40() {
checkAnalyze(ClickBenchUtils.Q40);
}
@Test
public void q41() {
checkAnalyze(ClickBenchUtils.Q41);
}
@Test
public void q42() {
checkAnalyze(ClickBenchUtils.Q42);
}
}

View File

@ -0,0 +1,29 @@
// 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.datasets.clickbench;
import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase;
public class ClickBenchTestBase extends AnalyzeCheckTestBase {
@Override
protected void runBeforeAll() throws Exception {
createDatabase("test");
connectContext.setDatabase("default_cluster:test");
ClickBenchUtils.createTables(this);
}
}

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.datasets.clickbench;
import org.apache.doris.utframe.TestWithFeService;
public class ClickBenchUtils {
public static final String Q0 = "SELECT COUNT(*) FROM hits;";
public static final String Q1 = "SELECT COUNT(*) FROM hits WHERE AdvEngineID <> 0;";
public static final String Q2 = "SELECT SUM(AdvEngineID), COUNT(*), AVG(ResolutionWidth) FROM hits;";
public static final String Q3 = "SELECT AVG(UserID) FROM hits;";
public static final String Q4 = "SELECT COUNT(DISTINCT UserID) FROM hits;";
public static final String Q5 = "SELECT COUNT(DISTINCT SearchPhrase) FROM hits;";
public static final String Q6 = "SELECT MIN(EventDate), MAX(EventDate) FROM hits;";
public static final String Q7 = "SELECT AdvEngineID, COUNT(*) FROM hits WHERE AdvEngineID <> 0 GROUP BY AdvEngineID ORDER BY COUNT(*) DESC;";
public static final String Q8 = "SELECT RegionID, COUNT(DISTINCT UserID) AS u FROM hits GROUP BY RegionID ORDER BY u DESC LIMIT 10;";
public static final String Q9 = "SELECT RegionID, SUM(AdvEngineID), COUNT(*) AS c, AVG(ResolutionWidth), COUNT(DISTINCT UserID) FROM hits GROUP BY RegionID ORDER BY c DESC LIMIT 10;";
public static final String Q10 = "SELECT MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhoneModel ORDER BY u DESC LIMIT 10;";
public static final String Q11 = "SELECT MobilePhone, MobilePhoneModel, COUNT(DISTINCT UserID) AS u FROM hits WHERE MobilePhoneModel <> '' GROUP BY MobilePhone, MobilePhoneModel ORDER BY u DESC LIMIT 10;";
public static final String Q12 = "SELECT SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;";
public static final String Q13 = "SELECT SearchPhrase, COUNT(DISTINCT UserID) AS u FROM hits WHERE SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY u DESC LIMIT 10;";
public static final String Q14 = "SELECT SearchEngineID, SearchPhrase, COUNT(*) AS c FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, SearchPhrase ORDER BY c DESC LIMIT 10;";
public static final String Q15 = "SELECT UserID, COUNT(*) FROM hits GROUP BY UserID ORDER BY COUNT(*) DESC LIMIT 10;";
public static final String Q16 = "SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10;";
public static final String Q17 = "SELECT UserID, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, SearchPhrase LIMIT 10;";
public static final String Q18 = "SELECT UserID, extract(minute FROM EventTime) AS m, SearchPhrase, COUNT(*) FROM hits GROUP BY UserID, m, SearchPhrase ORDER BY COUNT(*) DESC LIMIT 10;";
public static final String Q19 = "SELECT UserID FROM hits WHERE UserID = 435090932899640449;";
public static final String Q20 = "SELECT COUNT(*) FROM hits WHERE URL LIKE '%google%';";
public static final String Q21 = "SELECT SearchPhrase, MIN(URL), COUNT(*) AS c FROM hits WHERE URL LIKE '%google%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;";
public static final String Q22 = "SELECT SearchPhrase, MIN(URL), MIN(Title), COUNT(*) AS c, COUNT(DISTINCT UserID) FROM hits WHERE Title LIKE '%Google%' AND URL NOT LIKE '%.google.%' AND SearchPhrase <> '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;";
public static final String Q23 = "SELECT * FROM hits WHERE URL LIKE '%google%' ORDER BY EventTime LIMIT 10;";
public static final String Q24 = "SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime LIMIT 10;";
public static final String Q25 = "SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY SearchPhrase LIMIT 10;";
public static final String Q26 = "SELECT SearchPhrase FROM hits WHERE SearchPhrase <> '' ORDER BY EventTime, SearchPhrase LIMIT 10;";
public static final String Q27 = "SELECT CounterID, AVG(length(URL)) AS l, COUNT(*) AS c FROM hits WHERE URL <> '' GROUP BY CounterID HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25;";
public static final String Q28 = "SELECT REGEXP_REPLACE(Referer, '^https?://(?:www\\.)?([^/]+)/.*$', '\\\\1') AS k, AVG(length(Referer)) AS l, COUNT(*) AS c, MIN(Referer) FROM hits WHERE Referer <> '' GROUP BY k HAVING COUNT(*) > 100000 ORDER BY l DESC LIMIT 25;";
public static final String Q29 = "SELECT SUM(ResolutionWidth), SUM(ResolutionWidth + 1), SUM(ResolutionWidth + 2), SUM(ResolutionWidth + 3), SUM(ResolutionWidth + 4), SUM(ResolutionWidth + 5), SUM(ResolutionWidth + 6), SUM(ResolutionWidth + 7), SUM(ResolutionWidth + 8), SUM(ResolutionWidth + 9), SUM(ResolutionWidth + 10), SUM(ResolutionWidth + 11), SUM(ResolutionWidth + 12), SUM(ResolutionWidth + 13), SUM(ResolutionWidth + 14), SUM(ResolutionWidth + 15), SUM(ResolutionWidth + 16), SUM(ResolutionWidth + 17), SUM(ResolutionWidth + 18), SUM(ResolutionWidth + 19), SUM(ResolutionWidth + 20), SUM(ResolutionWidth + 21), SUM(ResolutionWidth + 22), SUM(ResolutionWidth + 23), SUM(ResolutionWidth + 24), SUM(ResolutionWidth + 25), SUM(ResolutionWidth + 26), SUM(ResolutionWidth + 27), SUM(ResolutionWidth + 28), SUM(ResolutionWidth + 29), SUM(ResolutionWidth + 30), SUM(ResolutionWidth + 31), SUM(ResolutionWidth + 32), SUM(ResolutionWidth + 33), SUM(ResolutionWidth + 34), SUM(ResolutionWidth + 35), SUM(ResolutionWidth + 36), SUM(ResolutionWidth + 37), SUM(ResolutionWidth + 38), SUM(ResolutionWidth + 39), SUM(ResolutionWidth + 40), SUM(ResolutionWidth + 41), SUM(ResolutionWidth + 42), SUM(ResolutionWidth + 43), SUM(ResolutionWidth + 44), SUM(ResolutionWidth + 45), SUM(ResolutionWidth + 46), SUM(ResolutionWidth + 47), SUM(ResolutionWidth + 48), SUM(ResolutionWidth + 49), SUM(ResolutionWidth + 50), SUM(ResolutionWidth + 51), SUM(ResolutionWidth + 52), SUM(ResolutionWidth + 53), SUM(ResolutionWidth + 54), SUM(ResolutionWidth + 55), SUM(ResolutionWidth + 56), SUM(ResolutionWidth + 57), SUM(ResolutionWidth + 58), SUM(ResolutionWidth + 59), SUM(ResolutionWidth + 60), SUM(ResolutionWidth + 61), SUM(ResolutionWidth + 62), SUM(ResolutionWidth + 63), SUM(ResolutionWidth + 64), SUM(ResolutionWidth + 65), SUM(ResolutionWidth + 66), SUM(ResolutionWidth + 67), SUM(ResolutionWidth + 68), SUM(ResolutionWidth + 69), SUM(ResolutionWidth + 70), SUM(ResolutionWidth + 71), SUM(ResolutionWidth + 72), SUM(ResolutionWidth + 73), SUM(ResolutionWidth + 74), SUM(ResolutionWidth + 75), SUM(ResolutionWidth + 76), SUM(ResolutionWidth + 77), SUM(ResolutionWidth + 78), SUM(ResolutionWidth + 79), SUM(ResolutionWidth + 80), SUM(ResolutionWidth + 81), SUM(ResolutionWidth + 82), SUM(ResolutionWidth + 83), SUM(ResolutionWidth + 84), SUM(ResolutionWidth + 85), SUM(ResolutionWidth + 86), SUM(ResolutionWidth + 87), SUM(ResolutionWidth + 88), SUM(ResolutionWidth + 89) FROM hits;";
public static final String Q30 = "SELECT SearchEngineID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY SearchEngineID, ClientIP ORDER BY c DESC LIMIT 10;";
public static final String Q31 = "SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits WHERE SearchPhrase <> '' GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;";
public static final String Q32 = "SELECT WatchID, ClientIP, COUNT(*) AS c, SUM(IsRefresh), AVG(ResolutionWidth) FROM hits GROUP BY WatchID, ClientIP ORDER BY c DESC LIMIT 10;";
public static final String Q33 = "SELECT URL, COUNT(*) AS c FROM hits GROUP BY URL ORDER BY c DESC LIMIT 10;";
public static final String Q34 = "SELECT 1, URL, COUNT(*) AS c FROM hits GROUP BY 1, URL ORDER BY c DESC LIMIT 10;";
public static final String Q35 = "SELECT ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3, COUNT(*) AS c FROM hits GROUP BY ClientIP, ClientIP - 1, ClientIP - 2, ClientIP - 3 ORDER BY c DESC LIMIT 10;";
public static final String Q36 = "SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND URL <> '' GROUP BY URL ORDER BY PageViews DESC LIMIT 10;";
public static final String Q37 = "SELECT Title, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND DontCountHits = 0 AND IsRefresh = 0 AND Title <> '' GROUP BY Title ORDER BY PageViews DESC LIMIT 10;";
public static final String Q38 = "SELECT URL, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND IsLink <> 0 AND IsDownload = 0 GROUP BY URL ORDER BY PageViews DESC LIMIT 10 OFFSET 1000;";
public static final String Q39 = "SELECT TraficSourceID, SearchEngineID, AdvEngineID, CASE WHEN (SearchEngineID = 0 AND AdvEngineID = 0) THEN Referer ELSE '' END AS Src, URL AS Dst, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 GROUP BY TraficSourceID, SearchEngineID, AdvEngineID, Src, Dst ORDER BY PageViews DESC LIMIT 10 OFFSET 1000;";
public static final String Q40 = "SELECT URLHash, EventDate, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND TraficSourceID IN (-1, 6) AND RefererHash = 3594120000172545465 GROUP BY URLHash, EventDate ORDER BY PageViews DESC LIMIT 10 OFFSET 100;";
public static final String Q41 = "SELECT WindowClientWidth, WindowClientHeight, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-01' AND EventDate <= '2013-07-31' AND IsRefresh = 0 AND DontCountHits = 0 AND URLHash = 2868770270353813622 GROUP BY WindowClientWidth, WindowClientHeight ORDER BY PageViews DESC LIMIT 10 OFFSET 10000;";
public static final String Q42 = "SELECT DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') AS M, COUNT(*) AS PageViews FROM hits WHERE CounterID = 62 AND EventDate >= '2013-07-14' AND EventDate <= '2013-07-15' AND IsRefresh = 0 AND DontCountHits = 0 GROUP BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') ORDER BY DATE_FORMAT(EventTime, '%Y-%m-%d %H:%i:00') LIMIT 10 OFFSET 1000;";
public static void createTables(TestWithFeService service) throws Exception {
service.createTable("CREATE TABLE IF NOT EXISTS hits (\n"
+ "CounterID INT NOT NULL,\n"
+ "EventDate DATEV2 NOT NULL,\n"
+ "UserID BIGINT NOT NULL,\n"
+ "EventTime DATETIMEV2 NOT NULL,\n"
+ "WatchID BIGINT NOT NULL,\n"
+ "JavaEnable SMALLINT NOT NULL,\n"
+ "Title STRING NOT NULL,\n"
+ "GoodEvent SMALLINT NOT NULL,\n"
+ "ClientIP INT NOT NULL,\n"
+ "RegionID INT NOT NULL,\n"
+ "CounterClass SMALLINT NOT NULL,\n"
+ "OS SMALLINT NOT NULL,\n"
+ "UserAgent SMALLINT NOT NULL,\n"
+ "URL STRING NOT NULL,\n"
+ "Referer STRING NOT NULL,\n"
+ "IsRefresh SMALLINT NOT NULL,\n"
+ "RefererCategoryID SMALLINT NOT NULL,\n"
+ "RefererRegionID INT NOT NULL,\n"
+ "URLCategoryID SMALLINT NOT NULL,\n"
+ "URLRegionID INT NOT NULL,\n"
+ "ResolutionWidth SMALLINT NOT NULL,\n"
+ "ResolutionHeight SMALLINT NOT NULL,\n"
+ "ResolutionDepth SMALLINT NOT NULL,\n"
+ "FlashMajor SMALLINT NOT NULL,\n"
+ "FlashMinor SMALLINT NOT NULL,\n"
+ "FlashMinor2 STRING NOT NULL,\n"
+ "NetMajor SMALLINT NOT NULL,\n"
+ "NetMinor SMALLINT NOT NULL,\n"
+ "UserAgentMajor SMALLINT NOT NULL,\n"
+ "UserAgentMinor VARCHAR(255) NOT NULL,\n"
+ "CookieEnable SMALLINT NOT NULL,\n"
+ "JavascriptEnable SMALLINT NOT NULL,\n"
+ "IsMobile SMALLINT NOT NULL,\n"
+ "MobilePhone SMALLINT NOT NULL,\n"
+ "MobilePhoneModel STRING NOT NULL,\n"
+ "Params STRING NOT NULL,\n"
+ "IPNetworkID INT NOT NULL,\n"
+ "TraficSourceID SMALLINT NOT NULL,\n"
+ "SearchEngineID SMALLINT NOT NULL,\n"
+ "SearchPhrase STRING NOT NULL,\n"
+ "AdvEngineID SMALLINT NOT NULL,\n"
+ "IsArtifical SMALLINT NOT NULL,\n"
+ "WindowClientWidth SMALLINT NOT NULL,\n"
+ "WindowClientHeight SMALLINT NOT NULL,\n"
+ "ClientTimeZone SMALLINT NOT NULL,\n"
+ "ClientEventTime DATETIMEV2 NOT NULL,\n"
+ "SilverlightVersion1 SMALLINT NOT NULL,\n"
+ "SilverlightVersion2 SMALLINT NOT NULL,\n"
+ "SilverlightVersion3 INT NOT NULL,\n"
+ "SilverlightVersion4 SMALLINT NOT NULL,\n"
+ "PageCharset STRING NOT NULL,\n"
+ "CodeVersion INT NOT NULL,\n"
+ "IsLink SMALLINT NOT NULL,\n"
+ "IsDownload SMALLINT NOT NULL,\n"
+ "IsNotBounce SMALLINT NOT NULL,\n"
+ "FUniqID BIGINT NOT NULL,\n"
+ "OriginalURL STRING NOT NULL,\n"
+ "HID INT NOT NULL,\n"
+ "IsOldCounter SMALLINT NOT NULL,\n"
+ "IsEvent SMALLINT NOT NULL,\n"
+ "IsParameter SMALLINT NOT NULL,\n"
+ "DontCountHits SMALLINT NOT NULL,\n"
+ "WithHash SMALLINT NOT NULL,\n"
+ "HitColor CHAR NOT NULL,\n"
+ "LocalEventTime DATETIMEV2 NOT NULL,\n"
+ "Age SMALLINT NOT NULL,\n"
+ "Sex SMALLINT NOT NULL,\n"
+ "Income SMALLINT NOT NULL,\n"
+ "Interests SMALLINT NOT NULL,\n"
+ "Robotness SMALLINT NOT NULL,\n"
+ "RemoteIP INT NOT NULL,\n"
+ "WindowName INT NOT NULL,\n"
+ "OpenerName INT NOT NULL,\n"
+ "HistoryLength SMALLINT NOT NULL,\n"
+ "BrowserLanguage STRING NOT NULL,\n"
+ "BrowserCountry STRING NOT NULL,\n"
+ "SocialNetwork STRING NOT NULL,\n"
+ "SocialAction STRING NOT NULL,\n"
+ "HTTPError SMALLINT NOT NULL,\n"
+ "SendTiming INT NOT NULL,\n"
+ "DNSTiming INT NOT NULL,\n"
+ "ConnectTiming INT NOT NULL,\n"
+ "ResponseStartTiming INT NOT NULL,\n"
+ "ResponseEndTiming INT NOT NULL,\n"
+ "FetchTiming INT NOT NULL,\n"
+ "SocialSourceNetworkID SMALLINT NOT NULL,\n"
+ "SocialSourcePage STRING NOT NULL,\n"
+ "ParamPrice BIGINT NOT NULL,\n"
+ "ParamOrderID STRING NOT NULL,\n"
+ "ParamCurrency STRING NOT NULL,\n"
+ "ParamCurrencyID SMALLINT NOT NULL,\n"
+ "OpenstatServiceName STRING NOT NULL,\n"
+ "OpenstatCampaignID STRING NOT NULL,\n"
+ "OpenstatAdID STRING NOT NULL,\n"
+ "OpenstatSourceID STRING NOT NULL,\n"
+ "UTMSource STRING NOT NULL,\n"
+ "UTMMedium STRING NOT NULL,\n"
+ "UTMCampaign STRING NOT NULL,\n"
+ "UTMContent STRING NOT NULL,\n"
+ "UTMTerm STRING NOT NULL,\n"
+ "FromTag STRING NOT NULL,\n"
+ "HasGCLID SMALLINT NOT NULL,\n"
+ "RefererHash BIGINT NOT NULL,\n"
+ "URLHash BIGINT NOT NULL,\n"
+ "CLID INT NOT NULL\n"
+ ")\n"
+ "DUPLICATE KEY (CounterID, EventDate, UserID, EventTime, WatchID)\n"
+ "DISTRIBUTED BY HASH(UserID) BUCKETS 48\n"
+ "PROPERTIES ( \"replication_num\"=\"1\");"
);
}
}

View File

@ -20,6 +20,7 @@ package org.apache.doris.nereids.parser;
import org.apache.doris.common.ExceptionChecker;
import org.apache.doris.nereids.datasets.tpch.AnalyzeCheckTestBase;
import org.apache.doris.nereids.exceptions.AnalysisException;
import org.apache.doris.nereids.properties.OrderKey;
import org.apache.doris.nereids.rules.expression.rewrite.ExpressionRewrite;
import org.apache.doris.nereids.rules.expression.rewrite.rules.TypeCoercion;
import org.apache.doris.nereids.trees.expressions.Add;
@ -28,7 +29,6 @@ import org.apache.doris.nereids.trees.expressions.And;
import org.apache.doris.nereids.trees.expressions.Cast;
import org.apache.doris.nereids.trees.expressions.ExprId;
import org.apache.doris.nereids.trees.expressions.GreaterThan;
import org.apache.doris.nereids.trees.expressions.NamedExpressionUtil;
import org.apache.doris.nereids.trees.expressions.SlotReference;
import org.apache.doris.nereids.trees.expressions.functions.agg.Count;
import org.apache.doris.nereids.trees.expressions.functions.agg.Min;
@ -48,12 +48,12 @@ import org.junit.jupiter.api.Test;
import java.util.stream.Collectors;
public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMatchSupported {
public class ResolveAggregateFunctionsTest extends AnalyzeCheckTestBase implements PatternMatchSupported {
@Override
public void runBeforeAll() throws Exception {
createDatabase("test_having");
connectContext.setDatabase("default_cluster:test_having");
createDatabase("test_resolve_aggregate_functions");
connectContext.setDatabase("default_cluster:test_resolve_aggregate_functions");
createTables(
"CREATE TABLE t1 (\n"
+ " pk TINYINT,\n"
@ -78,17 +78,12 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
);
}
@Override
public void runBeforeEach() throws Exception {
NamedExpressionUtil.clear();
}
@Test
public void testHavingGroupBySlot() throws Exception {
public void testHavingGroupBySlot() {
String sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING a1 > 0";
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
@ -98,12 +93,11 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1)))
).when(FieldChecker.check("predicates", new GreaterThan(a1, new TinyIntLiteral((byte) 0))))));
NamedExpressionUtil.clear();
sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING a1 > 0";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias value = new Alias(new ExprId(3), a1, "value");
PlanChecker.from(connectContext).analyze(sql)
@ -115,7 +109,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value)))
).when(FieldChecker.check("predicates", new GreaterThan(value.toSlot(), new TinyIntLiteral((byte) 0))))));
NamedExpressionUtil.clear();
sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING value > 0";
PlanChecker.from(connectContext).analyze(sql)
@ -127,16 +120,15 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(value)))
).when(FieldChecker.check("predicates", new GreaterThan(value.toSlot(), new TinyIntLiteral((byte) 0))))));
NamedExpressionUtil.clear();
sql = "SELECT SUM(a2) FROM t1 GROUP BY a1 HAVING a1 > 0";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)");
PlanChecker.from(connectContext).analyze(sql)
@ -149,19 +141,18 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(sumA2, a1)))
).when(FieldChecker.check("predicates", new GreaterThan(a1, new TinyIntLiteral((byte) 0))))
).when(FieldChecker.check("projects", Lists.newArrayList(sumA2.toSlot()))));
NamedExpressionUtil.clear();
}
@Test
public void testHavingAggregateFunction() throws Exception {
public void testHavingAggregateFunction() {
String sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING SUM(a2) > 0";
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)");
PlanChecker.from(connectContext).analyze(sql)
@ -173,7 +164,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))
).when(FieldChecker.check("predicates", new GreaterThan(sumA2.toSlot(), Literal.of(0L))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot()))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING SUM(a2) > 0";
sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)");
@ -185,16 +175,15 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))
).when(FieldChecker.check("predicates", new GreaterThan(sumA2.toSlot(), Literal.of(0L))))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING SUM(a2) > 0";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias value = new Alias(new ExprId(3), new Sum(a2), "value");
PlanChecker.from(connectContext).analyze(sql)
@ -205,7 +194,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))
).when(FieldChecker.check("predicates", new GreaterThan(value.toSlot(), Literal.of(0L))))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING value > 0";
PlanChecker.from(connectContext).analyze(sql)
@ -216,21 +204,20 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))
).when(FieldChecker.check("predicates", new GreaterThan(value.toSlot(), Literal.of(0L))))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING MIN(pk) > 0";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)");
SlotReference pk = new SlotReference(
new ExprId(0), "pk", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)");
PlanChecker.from(connectContext).analyze(sql)
@ -242,7 +229,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK)))
).when(FieldChecker.check("predicates", new GreaterThan(minPK.toSlot(), Literal.of((byte) 0))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot()))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2) > 0";
Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))");
@ -254,7 +240,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2)))
).when(FieldChecker.check("predicates", new GreaterThan(sumA1A2.toSlot(), Literal.of(0L))))));
NamedExpressionUtil.clear();
sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2 + 3) > 0";
Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new SmallIntLiteral((short) 3))),
@ -268,7 +253,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23)))
).when(FieldChecker.check("predicates", new GreaterThan(sumA1A23.toSlot(), Literal.of(0L))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA1A2.toSlot()))));
NamedExpressionUtil.clear();
sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING COUNT(*) > 0";
Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)");
@ -281,23 +265,22 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar)))
).when(FieldChecker.check("predicates", new GreaterThan(countStar.toSlot(), Literal.of(0L))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot()))));
NamedExpressionUtil.clear();
}
@Test
void testJoin() throws Exception {
void testJoinWithHaving() {
String sql = "SELECT a1, sum(a2) FROM t1, t2 WHERE t1.pk = t2.pk GROUP BY a1 HAVING a1 > SUM(b1)";
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference b1 = new SlotReference(
new ExprId(4), "b1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t2")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t2")
);
Alias sumA2 = new Alias(new ExprId(6), new Sum(a2), "sum(a2)");
Alias sumB1 = new Alias(new ExprId(7), new Sum(b1), "sum(b1)");
@ -316,7 +299,6 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
).when(FieldChecker.check("predicates", new GreaterThan(new Cast(a1, BigIntType.INSTANCE),
sumB1.toSlot())))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot()))));
NamedExpressionUtil.clear();
}
@Test
@ -344,21 +326,21 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
}
@Test
void testComplexQuery() throws Exception {
void testComplexQueryWithHaving() {
String sql = "SELECT t1.pk + 1, t1.pk + 1 + 1, t1.pk + 2, SUM(a1), COUNT(a1) + 1, SUM(a1 + a2), COUNT(a2) as v1\n"
+ "FROM t1, t2 WHERE t1.pk = t2.pk GROUP BY t1.pk, t1.pk + 1\n"
+ "HAVING t1.pk > 0 AND COUNT(a1) + 1 > 0 AND SUM(a1 + a2) + 1 > 0 AND v1 + 1 > 0 AND v1 > 0";
SlotReference pk = new SlotReference(
new ExprId(0), "pk", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_having", "t1")
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias pk1 = new Alias(new ExprId(6), new Add(pk, Literal.of((byte) 1)), "(pk + 1)");
Alias pk11 = new Alias(new ExprId(7), new Add(new Add(pk, Literal.of((byte) 1)), Literal.of((short) 1)), "((pk + 1) + 1)");
@ -398,6 +380,172 @@ public class HavingClauseTest extends AnalyzeCheckTestBase implements PatternMat
pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1).stream()
.map(Alias::toSlot).collect(Collectors.toList()))
));
NamedExpressionUtil.clear();
}
@Test
public void testSortAggregateFunction() {
String sql = "SELECT a1 FROM t1 GROUP BY a1 ORDER BY SUM(a2)";
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot()))));
sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 ORDER BY SUM(a2)";
sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true))))));
sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 ORDER BY SUM(a2)";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias value = new Alias(new ExprId(3), new Sum(a2), "value");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, value)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA2.toSlot(), true, true))))));
sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 ORDER BY MIN(pk)";
a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)");
SlotReference pk = new SlotReference(
new ExprId(0), "pk", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA2, minPK)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(minPK.toSlot(), true, true))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA2.toSlot()))));
sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 ORDER BY SUM(a1 + a2)";
Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA1A2.toSlot(), true, true))))));
sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 ORDER BY SUM(a1 + a2 + 3)";
Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new SmallIntLiteral((short) 3))),
"sum(((a1 + a2) + 3))");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, sumA1A2, sumA1A23)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(sumA1A23.toSlot(), true, true))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot(), sumA1A2.toSlot()))));
sql = "SELECT a1 FROM t1 GROUP BY a1 ORDER BY COUNT(*)";
Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalOlapScan()
).when(FieldChecker.check("outputExpressions", Lists.newArrayList(a1, countStar)))
).when(FieldChecker.check("orderKeys", ImmutableList.of(new OrderKey(countStar.toSlot(), true, true))))
).when(FieldChecker.check("projects", Lists.newArrayList(a1.toSlot()))));
}
@Test
void testComplexQueryWithOrderBy() {
String sql = "SELECT t1.pk + 1, t1.pk + 1 + 1, t1.pk + 2, SUM(a1), COUNT(a1) + 1, SUM(a1 + a2), COUNT(a2) as v1\n"
+ "FROM t1, t2 WHERE t1.pk = t2.pk GROUP BY t1.pk, t1.pk + 1\n"
+ "ORDER BY t1.pk, COUNT(a1) + 1, SUM(a1 + a2) + 1, v1 + 1, v1";
SlotReference pk = new SlotReference(
new ExprId(0), "pk", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a1 = new SlotReference(
new ExprId(1), "a1", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
SlotReference a2 = new SlotReference(
new ExprId(2), "a2", TinyIntType.INSTANCE, true,
ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1")
);
Alias pk1 = new Alias(new ExprId(6), new Add(pk, Literal.of((byte) 1)), "(pk + 1)");
Alias pk11 = new Alias(new ExprId(7), new Add(new Add(pk, Literal.of((byte) 1)), Literal.of((short) 1)), "((pk + 1) + 1)");
Alias pk2 = new Alias(new ExprId(8), new Add(pk, Literal.of((byte) 2)), "(pk + 2)");
Alias sumA1 = new Alias(new ExprId(9), new Sum(a1), "SUM(a1)");
Alias countA11 = new Alias(new ExprId(10), new Add(new Count(a1), Literal.of(1L)), "(COUNT(a1) + 1)");
Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))");
Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1");
PlanChecker.from(connectContext).analyze(sql)
.matchesFromRoot(
logicalProject(
logicalSort(
logicalAggregate(
logicalFilter(
logicalJoin(
logicalOlapScan(),
logicalOlapScan()
)
)
).when(FieldChecker.check("outputExpressions",
Lists.newArrayList(pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1, pk)))
).when(FieldChecker.check("orderKeys",
ImmutableList.of(
new OrderKey(pk, true, true),
new OrderKey(countA11.toSlot(), true, true),
new OrderKey(new Add(sumA1A2.toSlot(), new TinyIntLiteral((byte) 1)), true, true),
new OrderKey(new Add(v1.toSlot(), new TinyIntLiteral((byte) 1)), true, true),
new OrderKey(v1.toSlot(), true, true)
)
))
).when(FieldChecker.check(
"projects", Lists.newArrayList(
pk1, pk11, pk2, sumA1, countA11, sumA1A2, v1).stream()
.map(Alias::toSlot).collect(Collectors.toList()))
));
}
}