Skip to content

Commit

Permalink
move validators into their own package
Browse files Browse the repository at this point in the history
also hides the visitor in internal classes to have a cleaner
public interface.
  • Loading branch information
mfussenegger committed Nov 27, 2014
1 parent 3adbad4 commit b109166
Show file tree
Hide file tree
Showing 5 changed files with 389 additions and 241 deletions.
249 changes: 8 additions & 241 deletions sql/src/main/java/io/crate/analyze/SelectStatementAnalyzer.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,9 +23,12 @@

import com.google.common.base.Optional;
import com.google.common.collect.Iterables;
import io.crate.analyze.validator.GroupBySymbolValidator;
import io.crate.analyze.validator.HavingSymbolValidator;
import io.crate.analyze.validator.SelectSymbolValidator;
import io.crate.analyze.validator.SortSymbolValidator;
import io.crate.exceptions.SQLParseException;
import io.crate.metadata.*;
import io.crate.metadata.table.TableInfo;
import io.crate.planner.symbol.*;
import io.crate.planner.symbol.Literal;
import io.crate.sql.tree.*;
Expand All @@ -35,17 +38,12 @@
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.List;
import java.util.Locale;

import static com.google.common.base.MoreObjects.firstNonNull;

public class SelectStatementAnalyzer extends DataStatementAnalyzer<SelectAnalyzedStatement> {

private final static AggregationSearcher AGGREGATION_SEARCHER = new AggregationSearcher();
private final static SortSymbolValidator SORT_SYMBOL_VALIDATOR = new SortSymbolValidator();
private final static GroupBySymbolValidator GROUP_BY_SYMBOL_VALIDATOR = new GroupBySymbolValidator();
private final static SelectSymbolValidator SELECT_SYMBOL_VALIDATOR = new SelectSymbolValidator();
private final static HavingSymbolValidator HAVING_SYMBOL_VALIDATOR = new HavingSymbolValidator();
private final ReferenceInfos referenceInfos;
private final Functions functions;
private final ReferenceResolver globalReferenceResolver;
Expand Down Expand Up @@ -125,9 +123,7 @@ protected Symbol visitQuerySpecification(QuerySpecification node, SelectAnalyzed
process(node.getSelect(), context);

// validate select symbols
for (Symbol symbol : context.outputSymbols()) {
SELECT_SYMBOL_VALIDATOR.process(symbol, new SelectSymbolValidator.SelectContext(context.selectFromFieldCache));
}
SelectSymbolValidator.validate(context.outputSymbols(), context.selectFromFieldCache);

if (!node.getGroupBy().isEmpty()) {
analyzeGroupBy(node.getGroupBy(), context);
Expand Down Expand Up @@ -158,11 +154,7 @@ protected Symbol visitQuerySpecification(QuerySpecification node, SelectAnalyzed

private void processHavingClause(Expression expression, SelectAnalyzedStatement context) {
Symbol havingQuery = process(expression, context);

// validate having symbols
HavingSymbolValidator.HavingContext havingContext = new HavingSymbolValidator.HavingContext(context.groupBy());
HAVING_SYMBOL_VALIDATOR.process(havingQuery, havingContext);

HavingSymbolValidator.validate(havingQuery, context.groupBy());
context.havingClause(havingQuery);
}

Expand Down Expand Up @@ -279,7 +271,7 @@ private void analyzeGroupBy(List<Expression> groupByExpressions, SelectAnalyzedS
List<Symbol> groupBy = new ArrayList<>(groupByExpressions.size());
for (Expression expression : groupByExpressions) {
Symbol s = symbolFromSelectOutputReferenceOrExpression(expression, context, "GROUP BY");
GROUP_BY_SYMBOL_VALIDATOR.process(s, null);
GroupBySymbolValidator.validate(s);
groupBy.add(s);
}
context.groupBy(groupBy);
Expand Down Expand Up @@ -314,7 +306,7 @@ private boolean isAggregate(Symbol s) {
protected Symbol visitSortItem(SortItem node, SelectAnalyzedStatement context) {
Expression sortKey = node.getSortKey();
Symbol sortSymbol = symbolFromSelectOutputReferenceOrExpression(sortKey, context, "ORDER BY");
SORT_SYMBOL_VALIDATOR.process(sortSymbol, new SortSymbolValidator.SortContext(context.table));
SortSymbolValidator.validate(sortSymbol, context.table.partitionedBy());
return sortSymbol;
}

Expand Down Expand Up @@ -348,234 +340,9 @@ public Void visitAggregation(Aggregation symbol, AggregationSearcherContext cont
}
}

/**
* validate that sortSymbols don't contain partition by columns
*/
static class SortSymbolValidator extends SymbolVisitor<SortSymbolValidator.SortContext, Void> {

static class SortContext {
private final TableInfo tableInfo;
private boolean inFunction;
public SortContext(TableInfo tableInfo) {
this.tableInfo = tableInfo;
this.inFunction = false;
}
}

@Override
public Void visitFunction(Function symbol, SortContext context) {
try {
if (context.inFunction == false
&& !DataTypes.PRIMITIVE_TYPES.contains(symbol.valueType())) {
throw new UnsupportedOperationException(
String.format(Locale.ENGLISH,
"Cannot ORDER BY '%s': invalid return type '%s'.",
SymbolFormatter.format(symbol),
symbol.valueType())
);
}

if (symbol.info().type() == FunctionInfo.Type.PREDICATE) {
throw new UnsupportedOperationException(String.format(
"%s predicate cannot be used in an ORDER BY clause", symbol.info().ident().name()));
}

context.inFunction = true;
for (Symbol arg : symbol.arguments()) {
process(arg, context);
}
} finally {
context.inFunction = false;
}
return null;
}

@Override
public Void visitReference(Reference symbol, SortContext context) {
if (context.tableInfo.partitionedBy().contains(symbol.info().ident().columnIdent())) {
throw new UnsupportedOperationException(
SymbolFormatter.format(
"cannot use partitioned column %s in ORDER BY clause",
symbol));
}
// if we are in a function, we do not need to check the data type.
// the function will do that for us.
if (!context.inFunction && !DataTypes.PRIMITIVE_TYPES.contains(symbol.info().type())) {
throw new UnsupportedOperationException(
String.format(Locale.ENGLISH,
"Cannot ORDER BY '%s': invalid data type '%s'.",
SymbolFormatter.format(symbol),
symbol.valueType())
);
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.ANALYZED) {
throw new UnsupportedOperationException(
String.format("Cannot ORDER BY '%s': sorting on analyzed/fulltext columns is not possible",
SymbolFormatter.format(symbol)));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.NO) {
throw new UnsupportedOperationException(
String.format("Cannot ORDER BY '%s': sorting on non-indexed columns is not possible",
SymbolFormatter.format(symbol)));
}
return null;
}

@Override
public Void visitDynamicReference(DynamicReference symbol, SortContext context) {
throw new UnsupportedOperationException(
SymbolFormatter.format("Cannot order by \"%s\". The column doesn't exist.", symbol));
}

@Override
public Void visitSymbol(Symbol symbol, SortContext context) {
return null;
}
}

static class GroupBySymbolValidator extends SymbolVisitor<Void, Void> {

@Override
public Void visitDynamicReference(DynamicReference symbol, Void context) {
throw new IllegalArgumentException(
SymbolFormatter.format("unknown column '%s' not allowed in GROUP BY", symbol));
}

@Override
public Void visitReference(Reference symbol, Void context) {
if (!DataTypes.PRIMITIVE_TYPES.contains(symbol.valueType())) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': invalid data type '%s'",
SymbolFormatter.format(symbol),
symbol.valueType()));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.ANALYZED) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': grouping on analyzed/fulltext columns is not possible",
SymbolFormatter.format(symbol)));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.NO) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': grouping on non-indexed columns is not possible",
SymbolFormatter.format(symbol)));
}
return null;
}

@Override
public Void visitFunction(Function symbol, Void context) {
switch (symbol.info().type()) {
case SCALAR:
break;
case AGGREGATE:
throw new IllegalArgumentException("Aggregate functions are not allowed in GROUP BY");
case PREDICATE:
throw new UnsupportedOperationException(String.format(
"%s predicate cannot be used in a GROUP BY clause", symbol.info().ident().name()));
default:
throw new UnsupportedOperationException(
String.format("FunctionInfo.Type %s not handled", symbol.info().type()));
}
return null;
}

@Override
protected Void visitSymbol(Symbol symbol, Void context) {
throw new UnsupportedOperationException(
String.format("Cannot GROUP BY for '%s'", SymbolFormatter.format(symbol))
);
}
}

static class HavingSymbolValidator extends SymbolVisitor<HavingSymbolValidator.HavingContext, Void> {
static class HavingContext {
private final Optional<List<Symbol>> groupBySymbols;

private boolean insideAggregation = false;

public HavingContext(@Nullable List<Symbol> groupBySymbols) {
this.groupBySymbols = Optional.fromNullable(groupBySymbols);
}
}

@Override
public Void visitReference(Reference symbol, HavingContext context) {
if (!context.insideAggregation && (!context.groupBySymbols.isPresent() || !context.groupBySymbols.get().contains(symbol)) ) {
throw new IllegalArgumentException(
SymbolFormatter.format("Cannot use reference %s outside of an Aggregation in HAVING clause. Only GROUP BY keys allowed here.", symbol));
}
return null;
}

@Override
public Void visitFunction(Function symbol, HavingContext context) {
if (symbol.info().type().equals(FunctionInfo.Type.AGGREGATE)) {
context.insideAggregation = true;
}
for (Symbol argument : symbol.arguments()) {
process(argument, context);
}
context.insideAggregation = false;
return null;
}

@Override
protected Void visitSymbol(Symbol symbol, HavingContext context) {
return null;
}
}

static class SelectSymbolValidator extends SymbolVisitor<SelectSymbolValidator.SelectContext, Void> {

static class SelectContext {
private boolean selectFromFieldCache;

public SelectContext(boolean selectFromFieldCache) {
this.selectFromFieldCache = selectFromFieldCache;
}
}


@Override
public Void visitReference(Reference symbol, SelectContext context) {
if (context.selectFromFieldCache) {
if (symbol.info().indexType() == ReferenceInfo.IndexType.ANALYZED) {
throw new IllegalArgumentException(
String.format("Cannot select analyzed column '%s' " +
"within grouping or aggregations",
SymbolFormatter.format(symbol)));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.NO) {
throw new IllegalArgumentException(
String.format("Cannot select non-indexed column '%s' " +
"within grouping or aggregations",
SymbolFormatter.format(symbol)));
}
}
return null;
}

@Override
public Void visitFunction(Function symbol, SelectContext context) {
switch (symbol.info().type()) {
case SCALAR:
break;
case AGGREGATE:
context.selectFromFieldCache = true;
break;
case PREDICATE:
throw new UnsupportedOperationException(String.format(
"%s predicate cannot be selected", symbol.info().ident().name()));
default:
throw new UnsupportedOperationException(String.format(
"FunctionInfo.Type %s not handled", symbol.info().type()));
}
for (Symbol arg : symbol.arguments()) {
process(arg, context);
}
return null;
}

@Override
public Void visitSymbol(Symbol symbol, SelectContext context) {
return null;
}

}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,87 @@
/*
* Licensed to CRATE Technology GmbH ("Crate") under one or more contributor
* license agreements. See the NOTICE file distributed with this work for
* additional information regarding copyright ownership. Crate 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.
*
* However, if you have executed another commercial license agreement
* with Crate these terms will supersede the license and you may use the
* software solely pursuant to the terms of the relevant commercial agreement.
*/

package io.crate.analyze.validator;

import io.crate.metadata.ReferenceInfo;
import io.crate.planner.symbol.*;
import io.crate.types.DataTypes;

public class GroupBySymbolValidator {

private final static InnerValidator INNER_VALIDATOR = new InnerValidator();

public static void validate(Symbol symbol) throws IllegalArgumentException, UnsupportedOperationException {
INNER_VALIDATOR.process(symbol, null);
}

private static class InnerValidator extends SymbolVisitor<Void, Void> {

@Override
public Void visitDynamicReference(DynamicReference symbol, Void context) {
throw new IllegalArgumentException(
SymbolFormatter.format("unknown column '%s' not allowed in GROUP BY", symbol));
}

@Override
public Void visitReference(Reference symbol, Void context) {
if (!DataTypes.PRIMITIVE_TYPES.contains(symbol.valueType())) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': invalid data type '%s'",
SymbolFormatter.format(symbol),
symbol.valueType()));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.ANALYZED) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': grouping on analyzed/fulltext columns is not possible",
SymbolFormatter.format(symbol)));
} else if (symbol.info().indexType() == ReferenceInfo.IndexType.NO) {
throw new IllegalArgumentException(
String.format("Cannot GROUP BY '%s': grouping on non-indexed columns is not possible",
SymbolFormatter.format(symbol)));
}
return null;
}

@Override
public Void visitFunction(Function symbol, Void context) {
switch (symbol.info().type()) {
case SCALAR:
break;
case AGGREGATE:
throw new IllegalArgumentException("Aggregate functions are not allowed in GROUP BY");
case PREDICATE:
throw new UnsupportedOperationException(String.format(
"%s predicate cannot be used in a GROUP BY clause", symbol.info().ident().name()));
default:
throw new UnsupportedOperationException(
String.format("FunctionInfo.Type %s not handled", symbol.info().type()));
}
return null;
}

@Override
protected Void visitSymbol(Symbol symbol, Void context) {
throw new UnsupportedOperationException(
String.format("Cannot GROUP BY for '%s'", SymbolFormatter.format(symbol))
);
}
}
}

0 comments on commit b109166

Please sign in to comment.