diff --git a/sql/src/main/java/io/crate/analyze/SelectStatementAnalyzer.java b/sql/src/main/java/io/crate/analyze/SelectStatementAnalyzer.java index e8fd85316521..cd637d0ca1ba 100644 --- a/sql/src/main/java/io/crate/analyze/SelectStatementAnalyzer.java +++ b/sql/src/main/java/io/crate/analyze/SelectStatementAnalyzer.java @@ -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.*; @@ -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 { 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; @@ -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); @@ -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); } @@ -279,7 +271,7 @@ private void analyzeGroupBy(List groupByExpressions, SelectAnalyzedS List 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); @@ -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; } @@ -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 { - - 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 { - - @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 { - static class HavingContext { - private final Optional> groupBySymbols; - - private boolean insideAggregation = false; - - public HavingContext(@Nullable List 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 { - - 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; - } - - } } diff --git a/sql/src/main/java/io/crate/analyze/validator/GroupBySymbolValidator.java b/sql/src/main/java/io/crate/analyze/validator/GroupBySymbolValidator.java new file mode 100644 index 000000000000..2ebffa7b54c0 --- /dev/null +++ b/sql/src/main/java/io/crate/analyze/validator/GroupBySymbolValidator.java @@ -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 { + + @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)) + ); + } + } +} \ No newline at end of file diff --git a/sql/src/main/java/io/crate/analyze/validator/HavingSymbolValidator.java b/sql/src/main/java/io/crate/analyze/validator/HavingSymbolValidator.java new file mode 100644 index 000000000000..3bfa2c034b9b --- /dev/null +++ b/sql/src/main/java/io/crate/analyze/validator/HavingSymbolValidator.java @@ -0,0 +1,78 @@ +/* + * 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.FunctionInfo; +import io.crate.planner.symbol.*; + +import javax.annotation.Nullable; +import java.util.List; + +public class HavingSymbolValidator { + + private final static InnerValidator INNER_VALIDATOR = new InnerValidator(); + + public static void validate(Symbol symbol, @Nullable List groupBySymbols) throws IllegalArgumentException { + INNER_VALIDATOR.process(symbol, new HavingContext(groupBySymbols)); + } + + static class HavingContext { + @Nullable + private final List groupBySymbols; + + private boolean insideAggregation = false; + + public HavingContext(@Nullable List groupBySymbols) { + this.groupBySymbols = groupBySymbols; + } + } + + private static class InnerValidator extends SymbolVisitor { + + @Override + public Void visitReference(Reference symbol, HavingContext context) { + if (!context.insideAggregation && (context.groupBySymbols == null || !context.groupBySymbols.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; + } + } +} \ No newline at end of file diff --git a/sql/src/main/java/io/crate/analyze/validator/SelectSymbolValidator.java b/sql/src/main/java/io/crate/analyze/validator/SelectSymbolValidator.java new file mode 100644 index 000000000000..6d209e17e3fa --- /dev/null +++ b/sql/src/main/java/io/crate/analyze/validator/SelectSymbolValidator.java @@ -0,0 +1,93 @@ +/* + * 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 java.util.List; + +public class SelectSymbolValidator { + + private final static InnerValidator INNER_VALIDATOR = new InnerValidator(); + + public static void validate(List symbols, boolean selectFromFieldCache) { + for (Symbol symbol : symbols) { + INNER_VALIDATOR.process(symbol, new SelectContext(selectFromFieldCache)); + } + } + + static class SelectContext { + private boolean selectFromFieldCache; + + public SelectContext(boolean selectFromFieldCache) { + this.selectFromFieldCache = selectFromFieldCache; + } + } + + private static class InnerValidator extends SymbolVisitor { + + @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; + } + } +} \ No newline at end of file diff --git a/sql/src/main/java/io/crate/analyze/validator/SortSymbolValidator.java b/sql/src/main/java/io/crate/analyze/validator/SortSymbolValidator.java new file mode 100644 index 000000000000..7e006ce55469 --- /dev/null +++ b/sql/src/main/java/io/crate/analyze/validator/SortSymbolValidator.java @@ -0,0 +1,123 @@ +/* + * 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.ColumnIdent; +import io.crate.metadata.FunctionInfo; +import io.crate.metadata.ReferenceInfo; +import io.crate.planner.symbol.*; +import io.crate.types.DataTypes; + +import java.util.List; +import java.util.Locale; + +/** + * validate that sortSymbols don't contain partition by columns + */ +public class SortSymbolValidator { + + private final static InnerValidator INNER_VALIDATOR = new InnerValidator(); + + public static void validate(Symbol symbol, List partitionedByColumns) throws UnsupportedOperationException { + INNER_VALIDATOR.process(symbol, new SortContext(partitionedByColumns)); + } + + static class SortContext { + + private boolean inFunction; + private List partitionedByColumns; + + public SortContext(List partitionedByColumns) { + this.partitionedByColumns = partitionedByColumns; + this.inFunction = false; + } + } + + private static class InnerValidator extends SymbolVisitor { + + @Override + public Void visitFunction(Function symbol, SortContext context) { + if (!context.inFunction && !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())); + } + + try { + 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.partitionedByColumns.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; + } + } +} \ No newline at end of file