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

Support aliases in PARTITION BY and GROUP BY #4952

Merged
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -29,6 +29,7 @@
import io.confluent.ksql.execution.plan.SelectExpression;
import io.confluent.ksql.function.FunctionRegistry;
import io.confluent.ksql.name.FunctionName;
import io.confluent.ksql.parser.tree.GroupBy;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SchemaUtil;
import java.util.ArrayList;
Expand All @@ -52,7 +53,7 @@ public AggregateAnalysisResult analyze(
final ImmutableAnalysis analysis,
final List<SelectExpression> finalProjection
) {
if (analysis.getGroupByExpressions().isEmpty()) {
if (!analysis.getGroupBy().isPresent()) {
throw new IllegalArgumentException("Not an aggregate query");
}

Expand Down Expand Up @@ -96,7 +97,9 @@ public void process(final List<SelectExpression> finalProjection) {
analysis.getWhereExpression()
.ifPresent(this::processWhere);

analysis.getGroupByExpressions()
analysis.getGroupBy()
.map(GroupBy::getGroupingExpressions)
.orElseGet(ImmutableList::of)
.forEach(this::processGroupBy);

analysis.getHavingExpression()
Expand Down Expand Up @@ -179,8 +182,12 @@ private void throwOnWindowBoundColumnIfWindowedAggregate(
private static Set<Expression> getGroupByExpressions(
final ImmutableAnalysis analysis
) {
final List<Expression> groupByExpressions = analysis.getGroupBy()
.map(GroupBy::getGroupingExpressions)
.orElseGet(ImmutableList::of);

if (!analysis.getWindowExpression().isPresent()) {
return ImmutableSet.copyOf(analysis.getGroupByExpressions());
return ImmutableSet.copyOf(groupByExpressions);
}

// Add in window bounds columns as implicit group by columns:
Expand All @@ -190,7 +197,7 @@ private static Set<Expression> getGroupByExpressions(
.collect(Collectors.toSet());

return ImmutableSet.<Expression>builder()
.addAll(analysis.getGroupByExpressions())
.addAll(groupByExpressions)
.addAll(windowBoundColumnRefs)
.build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.name.SourceName;
import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties;
import io.confluent.ksql.parser.tree.GroupBy;
import io.confluent.ksql.parser.tree.PartitionBy;
import io.confluent.ksql.parser.tree.ResultMaterialization;
import io.confluent.ksql.parser.tree.SelectItem;
import io.confluent.ksql.parser.tree.WindowExpression;
Expand All @@ -39,13 +41,11 @@
import io.confluent.ksql.planner.plan.JoinNode.JoinType;
import io.confluent.ksql.schema.ksql.LogicalSchema;
import io.confluent.ksql.serde.SerdeOption;
import io.confluent.ksql.util.KsqlException;
import io.confluent.ksql.util.SchemaUtil;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Objects;
Expand All @@ -65,9 +65,9 @@ public class Analysis implements ImmutableAnalysis {
private Optional<Expression> whereExpression = Optional.empty();
private final List<SelectItem> selectItems = new ArrayList<>();
private final Set<ColumnName> selectColumnNames = new HashSet<>();
private final Set<Expression> groupByExpressions = new LinkedHashSet<>();
private Optional<GroupBy> groupBy = Optional.empty();
private Optional<PartitionBy> partitionBy = Optional.empty();
private Optional<WindowExpression> windowExpression = Optional.empty();
private Optional<Expression> partitionBy = Optional.empty();
private Optional<Expression> havingExpression = Optional.empty();
private OptionalInt limitClause = OptionalInt.empty();
private CreateSourceAsProperties withProperties = CreateSourceAsProperties.none();
Expand Down Expand Up @@ -126,19 +126,6 @@ public Set<ColumnName> getSelectColumnNames() {
return Collections.unmodifiableSet(selectColumnNames);
}

@Override
public List<Expression> getGroupByExpressions() {
return ImmutableList.copyOf(groupByExpressions);
}

void setGroupByExpressions(final List<Expression> expressions) {
expressions.forEach(exp -> {
if (!groupByExpressions.add(exp)) {
throw new KsqlException("Duplicate GROUP BY expression: " + exp);
}
});
}

@Override
public Optional<WindowExpression> getWindowExpression() {
return windowExpression;
Expand All @@ -158,11 +145,20 @@ void setHavingExpression(final Expression havingExpression) {
}

@Override
public Optional<Expression> getPartitionBy() {
public Optional<GroupBy> getGroupBy() {
return groupBy;
}

void setGroupBy(final GroupBy groupBy) {
this.groupBy = Optional.of(groupBy);
}

@Override
public Optional<PartitionBy> getPartitionBy() {
return partitionBy;
}

void setPartitionBy(final Expression partitionBy) {
void setPartitionBy(final PartitionBy partitionBy) {
this.partitionBy = Optional.of(partitionBy);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -264,7 +264,9 @@ private void throwOnUnknownColumnReference() {
analysis.getWhereExpression()
.ifPresent(columnValidator::analyzeExpression);

analysis.getGroupByExpressions()
analysis.getGroupBy()
.map(GroupBy::getGroupingExpressions)
.orElseGet(ImmutableList::of)
.forEach(columnValidator::analyzeExpression);

analysis.getHavingExpression()
Expand Down Expand Up @@ -488,11 +490,11 @@ private void analyzeWhere(final Expression node) {

private void analyzeGroupBy(final GroupBy groupBy) {
isGroupBy = true;
analysis.setGroupByExpressions(groupBy.getGroupingExpressions()); // Todo(ac): Alias
analysis.setGroupBy(groupBy);
}

private void analyzePartitionBy(final PartitionBy partitionBy) {
analysis.setPartitionBy(partitionBy.getExpression()); // Todo(ac): Alias
analysis.setPartitionBy(partitionBy);
}

private void analyzeWindowExpression(final WindowExpression windowExpression) {
Expand All @@ -514,7 +516,7 @@ private void validateSelect(final SingleColumn column) {
}
}

if (analysis.getGroupByExpressions().isEmpty()) {
if (!analysis.getGroupBy().isPresent()) {
throwOnUdafs(column.getExpression());
}
}
Expand Down Expand Up @@ -608,7 +610,7 @@ public Void visitFunctionCall(final FunctionCall functionCall, final Void contex

tableFunctionName = Optional.of(functionName);

if (!analysis.getGroupByExpressions().isEmpty()) {
if (analysis.getGroupBy().isPresent()) {
throw new KsqlException("Table functions cannot be used with aggregations.");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import io.confluent.ksql.execution.expression.tree.FunctionCall;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties;
import io.confluent.ksql.parser.tree.GroupBy;
import io.confluent.ksql.parser.tree.PartitionBy;
import io.confluent.ksql.parser.tree.SelectItem;
import io.confluent.ksql.parser.tree.WindowExpression;
import io.confluent.ksql.testing.EffectivelyImmutable;
Expand All @@ -44,15 +46,15 @@ public interface ImmutableAnalysis {

Set<ColumnName> getSelectColumnNames();

List<Expression> getGroupByExpressions();

Optional<Expression> getHavingExpression();

Optional<WindowExpression> getWindowExpression();

ColumnReferenceExp getDefaultArgument();

Optional<Expression> getPartitionBy();
Optional<GroupBy> getGroupBy();

Optional<PartitionBy> getPartitionBy();

OptionalInt getLimitClause();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ public class PullQueryValidator implements QueryValidator {
"Pull queries don't support WINDOW clauses."
),
Rule.of(
analysis -> analysis.getGroupByExpressions().isEmpty(),
analysis -> !analysis.getGroupBy().isPresent(),
"Pull queries don't support GROUP BY clauses."
),
Rule.of(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import io.confluent.ksql.execution.expression.tree.UnqualifiedColumnReferenceExp;
import io.confluent.ksql.name.ColumnName;
import io.confluent.ksql.parser.properties.with.CreateSourceAsProperties;
import io.confluent.ksql.parser.tree.GroupBy;
import io.confluent.ksql.parser.tree.PartitionBy;
import io.confluent.ksql.parser.tree.SelectItem;
import io.confluent.ksql.parser.tree.SingleColumn;
import io.confluent.ksql.parser.tree.WindowExpression;
Expand Down Expand Up @@ -98,17 +100,10 @@ public Optional<Into> getInto() {
@Override
public Set<ColumnName> getSelectColumnNames() {
return original.getSelectColumnNames().stream()
.map(UnqualifiedColumnReferenceExp::new)
.map(this::rewrite)
.map(UnqualifiedColumnReferenceExp::getColumnName)
.collect(Collectors.toSet());
}

@Override
public List<Expression> getGroupByExpressions() {
return rewriteList(original.getGroupByExpressions());
}

@Override
public Optional<Expression> getHavingExpression() {
return rewriteOptional(original.getHavingExpression());
Expand All @@ -125,8 +120,25 @@ public ColumnReferenceExp getDefaultArgument() {
}

@Override
public Optional<Expression> getPartitionBy() {
return rewriteOptional(original.getPartitionBy());
public Optional<PartitionBy> getPartitionBy() {
return original.getPartitionBy()
.map(partitionBy -> new PartitionBy(
partitionBy.getLocation(),
rewrite(partitionBy.getExpression()),
partitionBy.getAlias()
.map(this::rewrite)
));
}

@Override
public Optional<GroupBy> getGroupBy() {
return original.getGroupBy()
.map(groupBy -> new GroupBy(
groupBy.getLocation(),
rewriteList(groupBy.getGroupingExpressions()),
groupBy.getAlias()
.map(this::rewrite)
));
}

@Override
Expand Down Expand Up @@ -179,4 +191,9 @@ private <T extends Expression> List<T> rewriteList(final List<T> expressions) {
private <T extends Expression> T rewrite(final T expression) {
return ExpressionTreeRewriter.rewriteWith(rewriter, expression);
}

private ColumnName rewrite(final ColumnName name) {
final UnqualifiedColumnReferenceExp colRef = new UnqualifiedColumnReferenceExp(name);
return ExpressionTreeRewriter.rewriteWith(rewriter, colRef).getColumnName();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
import static java.util.Objects.requireNonNull;

import io.confluent.ksql.engine.rewrite.ExpressionTreeRewriter.Context;
import io.confluent.ksql.execution.expression.tree.ColumnReferenceExp;
import io.confluent.ksql.execution.expression.tree.DereferenceExpression;
import io.confluent.ksql.execution.expression.tree.Expression;
import io.confluent.ksql.execution.expression.tree.QualifiedColumnReferenceExp;
Expand All @@ -33,12 +34,15 @@
import io.confluent.ksql.parser.tree.AstNode;
import io.confluent.ksql.parser.tree.AstVisitor;
import io.confluent.ksql.parser.tree.CreateStreamAsSelect;
import io.confluent.ksql.parser.tree.GroupBy;
import io.confluent.ksql.parser.tree.InsertInto;
import io.confluent.ksql.parser.tree.PartitionBy;
import io.confluent.ksql.parser.tree.Query;
import io.confluent.ksql.parser.tree.SingleColumn;
import io.confluent.ksql.parser.tree.Statement;
import io.confluent.ksql.schema.ksql.FormatOptions;
import io.confluent.ksql.util.KsqlException;
import java.util.List;
import java.util.Optional;
import java.util.function.BiFunction;
import java.util.function.Supplier;
Expand Down Expand Up @@ -70,7 +74,7 @@ private static final class RewriterPlugin extends
final MetaStore metaStore;
final DataSourceExtractor dataSourceExtractor;

private Supplier<ColumnName> aliasGenerator;
private final Supplier<ColumnName> aliasGenerator;

RewriterPlugin(final MetaStore metaStore, final DataSourceExtractor dataSourceExtractor) {
super(Optional.empty());
Expand Down Expand Up @@ -159,6 +163,56 @@ protected Optional<AstNode> visitSingleColumn(
);
}

@Override
protected Optional<AstNode> visitGroupBy(
final GroupBy node,
final StatementRewriter.Context<Void> context
) {
final List<Expression> groupingExpressions = node.getGroupingExpressions();

if (node.getAlias().isPresent()
&& groupingExpressions.size() == 1
&& groupingExpressions.get(0) instanceof ColumnReferenceExp) {

final ColumnName groupByColName = ((ColumnReferenceExp) groupingExpressions.get(0))
.getColumnName();

if (node.getAlias().get().equals(groupByColName)) {
// Alias is a no-op - remove it:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

just wondering, why not just go through with it?

return Optional.of(new GroupBy(
node.getLocation(),
groupingExpressions,
Optional.empty()
));
}
}
return super.visitGroupBy(node, context);
}

@Override
protected Optional<AstNode> visitPartitionBy(
final PartitionBy node,
final StatementRewriter.Context<Void> context
) {
if (node.getAlias().isPresent()
&& node.getExpression() instanceof ColumnReferenceExp) {

final ColumnName groupByColName = ((ColumnReferenceExp) node.getExpression())
.getColumnName();

if (node.getAlias().get().equals(groupByColName)) {
// Alias is a no-op - remove it:
return Optional.of(new PartitionBy(
node.getLocation(),
node.getExpression(),
Optional.empty()
));
}
}

return super.visitPartitionBy(node, context);
}

private DataSource getSource(
final SourceName name,
final Optional<NodeLocation> location
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -439,7 +439,8 @@ protected AstNode visitPartitionBy(final PartitionBy node, final C context) {
return result
.orElseGet(() -> new PartitionBy(
node.getLocation(),
processExpression(node.getExpression(), context)
processExpression(node.getExpression(), context),
node.getAlias()
));
}

Expand All @@ -454,7 +455,7 @@ protected AstNode visitGroupBy(final GroupBy node, final C context) {
.map(exp -> processExpression(exp, context))
.collect(Collectors.toList());

return new GroupBy(node.getLocation(), rewrittenGroupings);
return new GroupBy(node.getLocation(), rewrittenGroupings, node.getAlias());
}

@Override
Expand Down
Loading