[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:
@ -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()
|
||||
))
|
||||
));
|
||||
}
|
||||
}
|
||||
|
||||
@ -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)));
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
@ -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),
|
||||
|
||||
@ -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());
|
||||
})
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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);
|
||||
}
|
||||
}
|
||||
@ -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\");"
|
||||
);
|
||||
}
|
||||
}
|
||||
@ -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()))
|
||||
));
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user