Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

extract QueryAnalyzer from QueryEngine #432

Merged
merged 1 commit into from
Nov 3, 2017
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -330,7 +330,7 @@ public Query addInto(final Query query, final QuerySpecification querySpecificat

QuerySpecification newQuerySpecification = new QuerySpecification(
querySpecification.getSelect(),
Optional.of(intoTable),
intoTable,
querySpecification.getFrom(),
querySpecification.getWindowExpression(),
querySpecification.getWhere(),
Expand Down
70 changes: 6 additions & 64 deletions ksql-engine/src/main/java/io/confluent/ksql/QueryEngine.java
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,8 @@
package io.confluent.ksql;

import io.confluent.ksql.analyzer.AggregateAnalysis;
import io.confluent.ksql.analyzer.AggregateAnalyzer;
import io.confluent.ksql.analyzer.Analysis;
import io.confluent.ksql.analyzer.AnalysisContext;
import io.confluent.ksql.analyzer.Analyzer;
import io.confluent.ksql.analyzer.QueryAnalyzer;
import io.confluent.ksql.ddl.commands.CreateStreamCommand;
import io.confluent.ksql.ddl.commands.CreateTableCommand;
import io.confluent.ksql.ddl.commands.DDLCommand;
Expand All @@ -33,16 +31,12 @@
import io.confluent.ksql.metastore.KsqlTopic;
import io.confluent.ksql.metastore.MetaStore;
import io.confluent.ksql.metastore.StructuredDataSource;
import io.confluent.ksql.util.AggregateExpressionRewriter;
import io.confluent.ksql.parser.tree.CreateStream;
import io.confluent.ksql.parser.tree.CreateTable;
import io.confluent.ksql.parser.tree.DropStream;
import io.confluent.ksql.parser.tree.DropTable;
import io.confluent.ksql.parser.tree.DropTopic;
import io.confluent.ksql.parser.tree.Expression;
import io.confluent.ksql.parser.tree.ExpressionTreeRewriter;
import io.confluent.ksql.parser.tree.Query;
import io.confluent.ksql.parser.tree.QuerySpecification;
import io.confluent.ksql.parser.tree.RegisterTopic;
import io.confluent.ksql.parser.tree.Select;
import io.confluent.ksql.parser.tree.SelectItem;
Expand Down Expand Up @@ -104,50 +98,11 @@ List<Pair<String, PlanNode>> buildLogicalPlans(
}

private PlanNode buildQueryLogicalPlan(final Query query, final MetaStore tempMetaStore) {

// Analyze the query to resolve the references and extract operations
Analysis analysis = new Analysis();
Analyzer analyzer = new Analyzer(analysis, tempMetaStore);
analyzer.process(query, new AnalysisContext(null));

AggregateAnalysis aggregateAnalysis = new AggregateAnalysis();
AggregateAnalyzer aggregateAnalyzer = new
AggregateAnalyzer(aggregateAnalysis, analysis, ksqlEngine.getFunctionRegistry());
AggregateExpressionRewriter aggregateExpressionRewriter =
new AggregateExpressionRewriter(ksqlEngine.getFunctionRegistry());
for (Expression expression: analysis.getSelectExpressions()) {
aggregateAnalyzer
.process(expression, new AnalysisContext(null));
if (!aggregateAnalyzer.isHasAggregateFunction()) {
aggregateAnalysis.getNonAggResultColumns().add(expression);
}
aggregateAnalysis.getFinalSelectExpressions()
.add(ExpressionTreeRewriter.rewriteWith(aggregateExpressionRewriter, expression));
aggregateAnalyzer.setHasAggregateFunction(false);
}

if (!aggregateAnalysis.getAggregateFunctionArguments().isEmpty() &&
analysis.getGroupByExpressions().isEmpty()) {
throw new KsqlException("Aggregate query needs GROUP BY clause.");
}
// TODO: make sure only aggregates are in the expression. For now we assume this is the case.
if (analysis.getHavingExpression() != null) {
aggregateAnalyzer.process(analysis.getHavingExpression(),
new AnalysisContext(null));
if (!aggregateAnalyzer.isHasAggregateFunction()) {
aggregateAnalysis.getNonAggResultColumns().add(analysis.getHavingExpression());
}
aggregateAnalysis
.setHavingExpression(ExpressionTreeRewriter.rewriteWith(aggregateExpressionRewriter,
analysis.getHavingExpression()));
aggregateAnalyzer.setHasAggregateFunction(false);
}

enforceAggregateRules(query, aggregateAnalysis);


// Build a logical plan
PlanNode logicalPlan = new LogicalPlanner(analysis, aggregateAnalysis, ksqlEngine.getFunctionRegistry()).buildPlan();
final QueryAnalyzer queryAnalyzer = new QueryAnalyzer(tempMetaStore, ksqlEngine.getFunctionRegistry());
final Analysis analysis = queryAnalyzer.analyize(query);
final AggregateAnalysis aggAnalysis = queryAnalyzer.analyizeAggregate(query, analysis);
final PlanNode logicalPlan
= new LogicalPlanner(analysis, aggAnalysis, ksqlEngine.getFunctionRegistry()).buildPlan();
if (logicalPlan instanceof KsqlStructuredDataOutputNode) {
KsqlStructuredDataOutputNode ksqlStructuredDataOutputNode =
(KsqlStructuredDataOutputNode) logicalPlan;
Expand Down Expand Up @@ -268,17 +223,4 @@ StructuredDataSource getResultDatasource(final Select select, final String name)
KsqlTopic ksqlTopic = new KsqlTopic(name, name, null);
return new KsqlStream(name, dataSource.schema(), null, null, ksqlTopic);
}

private void enforceAggregateRules(Query query, AggregateAnalysis aggregateAnalysis) {
if (!((QuerySpecification) query.getQueryBody()).getGroupBy().isPresent()) {
return;
}
int numberOfNonAggProjections = aggregateAnalysis.getNonAggResultColumns().size();
int groupBySize = ((QuerySpecification) query.getQueryBody()).getGroupBy().get()
.getGroupingElements().size();
if (numberOfNonAggProjections != groupBySize) {
throw new KsqlException("Group by elements should match the SELECT expressions.");
}
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -142,5 +142,9 @@ public void setLimitClause(Optional<Integer> limitClause) {
public Pair<StructuredDataSource, String> getFromDataSource(int index) {
return fromDataSources.get(index);
}

void addDataSource(Pair<StructuredDataSource, String> fromDataSource) {
fromDataSources.add(fromDataSource);
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,9 @@ public String getValue() {

private final ParentType parentType;

public AnalysisContext() {
this(null);
}
public AnalysisContext(final ParentType parentType) {
this.parentType = parentType;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,10 +77,10 @@ public Analyzer(Analysis analysis, MetaStore metaStore) {
protected Node visitQuerySpecification(final QuerySpecification node,
final AnalysisContext context) {

process(node.getFrom().get(),
process(node.getFrom(),
new AnalysisContext(AnalysisContext.ParentType.FROM));

process(node.getInto().get(), new AnalysisContext(
process(node.getInto(), new AnalysisContext(
AnalysisContext.ParentType.INTO));
if (!(analysis.getInto() instanceof KsqlStdOut)) {
analyzeNonStdOutSink();
Expand Down Expand Up @@ -312,7 +312,7 @@ protected Node visitAliasedRelation(AliasedRelation node, AnalysisContext contex
new Pair<>(
structuredDataSource,
node.getAlias());
analysis.getFromDataSources().add(fromDataSource);
analysis.addDataSource(fromDataSource);
return node;
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
/**
* Copyright 2017 Confluent Inc.
*
* Licensed 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 io.confluent.ksql.analyzer;

import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.metastore.MetaStore;
import io.confluent.ksql.parser.tree.Expression;
import io.confluent.ksql.parser.tree.ExpressionTreeRewriter;
import io.confluent.ksql.parser.tree.Query;
import io.confluent.ksql.parser.tree.QuerySpecification;
import io.confluent.ksql.util.AggregateExpressionRewriter;
import io.confluent.ksql.util.KsqlException;

public class QueryAnalyzer {
private final MetaStore metaStore;
private final FunctionRegistry functionRegistry;

public QueryAnalyzer(final MetaStore metaStore, final FunctionRegistry functionRegistry) {
this.metaStore = metaStore;
this.functionRegistry = functionRegistry;
}

public Analysis analyize(final Query query) {
Analysis analysis = new Analysis();
Analyzer analyzer = new Analyzer(analysis, metaStore);
analyzer.process(query, new AnalysisContext());
return analysis;
}

public AggregateAnalysis analyizeAggregate(final Query query, final Analysis analysis) {
AggregateAnalysis aggregateAnalysis = new AggregateAnalysis();
AggregateAnalyzer aggregateAnalyzer = new
AggregateAnalyzer(aggregateAnalysis, analysis, functionRegistry);
AggregateExpressionRewriter aggregateExpressionRewriter =
new AggregateExpressionRewriter(functionRegistry);

processSelectExpressions(analysis,
aggregateAnalysis,
aggregateAnalyzer,
aggregateExpressionRewriter);

if (!aggregateAnalysis.getAggregateFunctionArguments().isEmpty() &&
analysis.getGroupByExpressions().isEmpty()) {
throw new KsqlException("Aggregate query needs GROUP BY clause. query:" + query);
}

// TODO: make sure only aggregates are in the expression. For now we assume this is the case.
if (analysis.getHavingExpression() != null) {
processHavingExpression(analysis,
aggregateAnalysis,
aggregateAnalyzer,
aggregateExpressionRewriter);
}

enforceAggregateRules(query, aggregateAnalysis);
return aggregateAnalysis;
}

private void processHavingExpression(final Analysis analysis,
final AggregateAnalysis aggregateAnalysis,
final AggregateAnalyzer aggregateAnalyzer,
final AggregateExpressionRewriter aggregateExpressionRewriter) {
aggregateAnalyzer.process(analysis.getHavingExpression(),
new AnalysisContext());
if (!aggregateAnalyzer.isHasAggregateFunction()) {
aggregateAnalysis.getNonAggResultColumns().add(analysis.getHavingExpression());
}
aggregateAnalysis
.setHavingExpression(ExpressionTreeRewriter.rewriteWith(aggregateExpressionRewriter,
analysis.getHavingExpression()));
aggregateAnalyzer.setHasAggregateFunction(false);
}

private void processSelectExpressions(final Analysis analysis,
final AggregateAnalysis aggregateAnalysis,
final AggregateAnalyzer aggregateAnalyzer,
final AggregateExpressionRewriter aggregateExpressionRewriter) {
for (Expression expression: analysis.getSelectExpressions()) {
aggregateAnalyzer
.process(expression, new AnalysisContext());
if (!aggregateAnalyzer.isHasAggregateFunction()) {
aggregateAnalysis.getNonAggResultColumns().add(expression);
}
aggregateAnalysis.getFinalSelectExpressions()
.add(ExpressionTreeRewriter.rewriteWith(aggregateExpressionRewriter, expression));
aggregateAnalyzer.setHasAggregateFunction(false);
}
}

private void enforceAggregateRules(final Query query, final AggregateAnalysis aggregateAnalysis) {
if (!((QuerySpecification) query.getQueryBody()).getGroupBy().isPresent()) {
return;
}
int numberOfNonAggProjections = aggregateAnalysis.getNonAggResultColumns().size();
int groupBySize = ((QuerySpecification) query.getQueryBody()).getGroupBy().get()
.getGroupingElements().size();
if (numberOfNonAggProjections != groupBySize) {
throw new KsqlException("Group by elements should match the SELECT expressions.");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -320,9 +320,8 @@ protected R visitSortItem(SortItem node, C context) {
@Override
protected R visitQuerySpecification(QuerySpecification node, C context) {
process(node.getSelect(), context);
if (node.getFrom().isPresent()) {
process(node.getFrom().get(), context);
}
process(node.getFrom(), context);

if (node.getWhere().isPresent()) {
process(node.getWhere().get(), context);
}
Expand Down
Loading