diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/main/java/org/finos/legend/engine/language/sql/grammar/to/SQLGrammarComposer.java b/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/main/java/org/finos/legend/engine/language/sql/grammar/to/SQLGrammarComposer.java index d876c26f475..b2baf28dcba 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/main/java/org/finos/legend/engine/language/sql/grammar/to/SQLGrammarComposer.java +++ b/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/main/java/org/finos/legend/engine/language/sql/grammar/to/SQLGrammarComposer.java @@ -93,7 +93,7 @@ public String visit(AliasedRelation val) @Override public String visit(AllColumns val) { - return "*"; + return val.prefix != null ? val.prefix + ".*" : "*"; } @Override diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/test/java/org/finos/legend/engine/language/sql/grammar/test/roundtrip/TestSQLRoundTrip.java b/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/test/java/org/finos/legend/engine/language/sql/grammar/test/roundtrip/TestSQLRoundTrip.java index 76b643e9467..eb67271d096 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/test/java/org/finos/legend/engine/language/sql/grammar/test/roundtrip/TestSQLRoundTrip.java +++ b/legend-engine-xts-sql/legend-engine-xt-sql-grammar/src/test/java/org/finos/legend/engine/language/sql/grammar/test/roundtrip/TestSQLRoundTrip.java @@ -36,6 +36,7 @@ public void testEmptyStatement() public void testSelectStar() { check("SELECT * FROM myTable"); + check("SELECT myTable.* FROM myTable"); } @Test diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/fromPure.pure b/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/fromPure.pure index aaf0acdb4f2..fc44739ee63 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/fromPure.pure +++ b/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/fromPure.pure @@ -311,13 +311,13 @@ function <> meta::external::query::sql::transformation::queryToP ^$final(aliases = $aliases); } -function <> meta::external::query::sql::transformation::queryToPure::extractAggregatesFromExpression(expression:meta::external::query::sql::metamodel::Expression[0..1]):meta::external::query::sql::metamodel::Expression[*] +function <> meta::external::query::sql::transformation::queryToPure::extractAggregatesFromExpression(expression:meta::external::query::sql::metamodel::Expression[0..1]):FunctionCall[*] { walk($expression, t | $t, f:FunctionCall[1] | if (isExpressionAggregate($f, false), | $f, | $f.arguments->map(a | $a->extractAggregatesFromExpression())) - ) + )->cast(@FunctionCall); } function <> meta::external::query::sql::transformation::queryToPure::extractUsedColumnNames(expression:meta::external::query::sql::metamodel::Expression[*]):String[*] @@ -493,48 +493,36 @@ function <> meta::external::query::sql::transformation::queryToP !extractNameFromExpression($h.expression, $context)->in($aggregateExpressionNames); ))->removeDuplicatesBy(c | $c->extractNameFromSingleColumn($context)); - let aggregations = $allAggregateColumns->map(column | - let name = extractNameFromSingleColumn($column, $context); - - let aggregateExpression = extractAggregatesFromExpression($column.expression); - - let functionCalls = $aggregateExpression->match([ - f:FunctionCall[1] | ensureAggregateFunction($f), - f:FunctionCall[1..*] | - // we can only support multiple aggs being used within a single expression if they use the case arguments - //e.g. case when sum(col) > 10 then max(col) else min(col) end - //to support more complex cases we need to extract the aggregates into separate columns - assertEquals(1, $f.arguments->removeDuplicates()->size(), 'unsupported aggregation'); - - $f->map(fc | ensureAggregateFunction($fc));, - e:meta::external::query::sql::metamodel::Expression[*] | fail('invalid aggregate expression'); - ])->cast(@FunctionCall); - - // we take the first as the map only cares about the argument and we ensure they are the same above - let aggregationFunctionCall = $functionCalls->at(0); - let processor = aggregateProcessor($aggregationFunctionCall.name); - let adjusted = $processor.preProcess($column.expression, $aggregationFunctionCall); + //for any aggregate where there is multiple aggregations used with different arguments, we split the aggregates to individual columns and then extend + let aggregateColumnExpresions = $allAggregateColumns->map(column | + let aggregateExpressions = extractAggregatesFromExpression($column.expression)->removeDuplicates(); + pair($column, list($aggregateExpressions)); + )->groupBy(x | $x.second.values->map(a | $a.arguments->first())->removeDuplicates()->size() > 1); - let mapExpression = processMapFunctionArgument($adjusted.map, $context); - let mapReturnType = $mapExpression.genericType.rawType->toOne(); - let mapFunctionType = functionType('row', TDSRow, PureOne, $mapReturnType, ZeroOne); - let mapLambda = lambda($mapFunctionType, $mapExpression); - - let aggLambda = processAggregationLambda($adjusted.aggregate, $mapReturnType, $context); - let aggReturnType = $aggLambda->functionReturnType(); + let standardAggregations = $aggregateColumnExpresions->get(false).values->map(p | + let column = $p.first; + let name = extractNameFromSingleColumn($column, $context); + createAggregation($column, $name, $p.second.values->cast(@FunctionCall)->at(0), $context); + ); - let typeParameters = [^GenericType(rawType = $mapReturnType), $aggReturnType]; + let extractedAggregations = $aggregateColumnExpresions->get(true).values.second.values->removeDuplicates()->map(agg | + createAggregation(^SingleColumn(expression = $agg), extractNameFromExpression($agg, $context), $agg, $context); + ); - let genericType = ^GenericType(rawType = meta::pure::tds::AggregateValue, typeArguments = $typeParameters); + let aggExtensions = $aggregateColumnExpresions->get(true).values->map(p | + let aggregateExpressions = $p.second.values; + let column = $p.first; + ^$column(expression = $p.first.expression->walk([ + f:FunctionCall[1] | + if ($f->in($aggregateExpressions), | ^QualifiedNameReference(name = ^QualifiedName(parts = $f->extractNameFromExpression($context))), | $f); + ])->toOne()); + ); - sfe(agg_String_1__FunctionDefinition_1__FunctionDefinition_1__AggregateValue_1_, $genericType, $typeParameters, [ - iv($name), iv($mapLambda), iv($aggLambda) - ]); - ); + let aggregations = $standardAggregations->concatenate($extractedAggregations); - let aggregationInstanceValues = iv($aggregations, ^GenericType(rawType = meta::pure::tds::AggregateValue, typeArguments = [^GenericType(rawType = Any), ^GenericType(rawType = Any)])); + let aggregationInstanceValues = iv($aggregations, ^GenericType(rawType = meta::pure::tds::AggregateValue, typeArguments = [^GenericType(rawType = Any), ^GenericType(rawType = Any)])); - if ($allGroupByColumns->isEmpty() && $aggregations->isEmpty(), + let groupByExpression = if ($allGroupByColumns->isEmpty() && $aggregations->isEmpty(), | $rename, | if ($allGroupByColumns->isNotEmpty() && $aggregations->isEmpty(), | @@ -542,6 +530,30 @@ function <> meta::external::query::sql::transformation::queryToP let restrict = processRestrict($allGroupByColumns, ^$context(expression = $rename)); appendTdsFunc($restrict, distinct_TabularDataSet_1__TabularDataSet_1_, []);, | appendTdsFunc($rename, groupBy_TabularDataSet_1__String_MANY__AggregateValue_MANY__TabularDataSet_1_, [list($allGroupByColumns), list($aggregationInstanceValues)]))); + + if ($aggExtensions->isNotEmpty(), | processExtend($aggExtensions, [], ^$context(expression = $groupByExpression)), | $groupByExpression); +} + +function <> meta::external::query::sql::transformation::queryToPure::createAggregation(column:SingleColumn[1], name:String[1], aggregationFunctionCall:FunctionCall[1], context:SqlTransformContext[1]):SimpleFunctionExpression[1] +{ + let processor = aggregateProcessor($aggregationFunctionCall.name); + let adjusted = $processor.preProcess($column.expression, $aggregationFunctionCall); + + let mapExpression = processMapFunctionArgument($adjusted.map, $context); + let mapReturnType = $mapExpression.genericType.rawType->toOne(); + let mapFunctionType = functionType('row', TDSRow, PureOne, $mapReturnType, ZeroOne); + let mapLambda = lambda($mapFunctionType, $mapExpression); + + let aggLambda = processAggregationLambda($adjusted.aggregate, $mapReturnType, $context); + let aggReturnType = $aggLambda->functionReturnType(); + + let typeParameters = [^GenericType(rawType = $mapReturnType), $aggReturnType]; + + let genericType = ^GenericType(rawType = meta::pure::tds::AggregateValue, typeArguments = $typeParameters); + + sfe(agg_String_1__FunctionDefinition_1__FunctionDefinition_1__AggregateValue_1_, $genericType, $typeParameters, [ + iv($name), iv($mapLambda), iv($aggLambda) + ]); } function <> meta::external::query::sql::transformation::queryToPure::processAggregateFunctionCall(functionCall:FunctionCall[1]):FunctionCall[1] @@ -640,11 +652,12 @@ function <> meta::external::query::sql::transformation::queryToP } //We currently need to ensure that all the functions used in group by are in fact aggregate functions. e.g. select count(*), pi() would not currently translate well to Pure. -function <> meta::external::query::sql::transformation::queryToPure::ensureAggregateFunction(functionCall:FunctionCall[1]):FunctionCall[1] +function <> meta::external::query::sql::transformation::queryToPure::ensureAggregateFunction(e:meta::external::query::sql::metamodel::Expression[1]):Boolean[1] { + assert($e->instanceOf(FunctionCall), 'unsupported aggregation'); + let functionCall = $e->cast(@FunctionCall); let processor = functionProcessor($functionCall.name); assert($processor.isAggregate, | 'function ' + extractNameFromQualifiedName($functionCall.name, []) + ' is not currently supported as aggregate function'); - $functionCall; } function <> meta::external::query::sql::transformation::queryToPure::processExtend(items: SelectItem[*], columnsToRealias:String[*], context: SqlTransformContext[1]):FunctionExpression[1] diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/tests/testTranspile.pure b/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/tests/testTranspile.pure index 07ec7ff5cd9..c20a3e4ba86 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/tests/testTranspile.pure +++ b/legend-engine-xts-sql/legend-engine-xt-sql-pure/src/main/resources/core_external_query_sql/binding/fromPure/tests/testTranspile.pure @@ -805,7 +805,14 @@ function <> meta::external::query::sql::transformation::queryToPure:: function <> meta::external::query::sql::transformation::queryToPure::tests::testGroupByAggregateExpressions():Boolean[1] { test( - 'SELECT Integer, sum(1) AS "sum", count(*) AS "count", sum(Integer + Float) AS "exp", EXTRACT(YEAR FROM MAX("StrictDate")) AS "aggFunctionCall", MAX("StrictDate") + INTERVAL \'1 WEEK\' AS "dateAggMath" FROM service."/service/service1" GROUP BY Integer', + 'SELECT ' + + 'Integer, ' + + 'sum(1) AS "sum", ' + + 'count(*) AS "count", ' + + 'sum(Integer + Float) AS "exp", ' + + 'EXTRACT(YEAR FROM MAX("StrictDate")) AS "aggFunctionCall", ' + + 'MAX("StrictDate") + INTERVAL \'1 WEEK\' AS "dateAggMath" ' + + 'FROM service."/service/service1" GROUP BY Integer', {| FlatInput.all()->project( [ x | $x.booleanIn, x | $x.integerIn, x | $x.floatIn, x | $x.decimalIn, x | $x.strictDateIn, x | $x.dateTimeIn, x | $x.stringIn ], @@ -815,8 +822,35 @@ function <> meta::external::query::sql::transformation::queryToPure:: agg('count', row | $row, y | $y->count()), agg('exp', row | $row.getInteger('Integer') + $row.getFloat('Float'), y | $y->sum()), agg('aggFunctionCall', row | $row.getStrictDate('StrictDate'), y | $y->max()->toOne()->year()), - agg('dateAggMath', row | $row.getStrictDate('StrictDate'), y | $y->max()->toOne()->adjust(1, DurationUnit.WEEKS)) + agg('dateAggMath', row | $row.getStrictDate('StrictDate'), y | $y->max()->toOne()->adjust(1, DurationUnit.WEEKS))]) + }, false) +} + +function <> meta::external::query::sql::transformation::queryToPure::tests::testGroupByAggregateMixedExpressions():Boolean[1] +{ + test( + 'SELECT ' + + 'Integer, ' + + 'sum(Integer) AS "sum", ' + + 'count(*) AS "count", ' + + 'CASE WHEN sum(Integer) > sum(Float) THEN 1 ELSE 0 END AS "multiArg", ' + + 'CASE WHEN sum(Integer) < count(Float) THEN 1 ELSE 0 END AS "multiArg2" ' + + 'FROM service."/service/service1" GROUP BY Integer', + + {| FlatInput.all()->project( + [ x | $x.booleanIn, x | $x.integerIn, x | $x.floatIn, x | $x.decimalIn, x | $x.strictDateIn, x | $x.dateTimeIn, x | $x.stringIn ], + [ 'Boolean', 'Integer', 'Float', 'Decimal', 'StrictDate', 'DateTime', 'String' ] + )->groupBy(['Integer'], [ + agg('sum', row | $row.getInteger('Integer'), y | $y->sum()), + agg('count', row | $row, y | $y->count()), + agg('sum(Integer)', row | $row.getInteger('Integer'), y | $y->sum()), + agg('sum(Float)', row | $row.getFloat('Float'), y | $y->sum()), + agg('count(Float)', row | $row.getFloat('Float'), y | $y->count()) + ])->extend([ + col(row:TDSRow[1] | if ($row.getInteger('sum(Integer)') > $row.getFloat('sum(Float)'), | 1, | 0), 'multiArg'), + col(row:TDSRow[1] | if ($row.getInteger('sum(Integer)') < $row.getInteger('count(Float)'), | 1, | 0), 'multiArg2') ]) + ->restrict(['Integer', 'sum', 'count', 'multiArg', 'multiArg2']) }, false) } diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/BaseNodeModifierVisitor.java b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/BaseNodeModifierVisitor.java new file mode 100644 index 00000000000..d1b766ae5b6 --- /dev/null +++ b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/BaseNodeModifierVisitor.java @@ -0,0 +1,519 @@ +// Copyright 2023 Goldman Sachs +// +// 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 org.finos.legend.engine.query.sql.api; + +import org.eclipse.collections.impl.utility.ListIterate; +import org.finos.legend.engine.protocol.sql.metamodel.AliasedRelation; +import org.finos.legend.engine.protocol.sql.metamodel.AllColumns; +import org.finos.legend.engine.protocol.sql.metamodel.ArithmeticExpression; +import org.finos.legend.engine.protocol.sql.metamodel.ArrayLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.BetweenPredicate; +import org.finos.legend.engine.protocol.sql.metamodel.BooleanLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.Cast; +import org.finos.legend.engine.protocol.sql.metamodel.ColumnType; +import org.finos.legend.engine.protocol.sql.metamodel.ComparisonExpression; +import org.finos.legend.engine.protocol.sql.metamodel.CurrentTime; +import org.finos.legend.engine.protocol.sql.metamodel.DoubleLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.Expression; +import org.finos.legend.engine.protocol.sql.metamodel.Extract; +import org.finos.legend.engine.protocol.sql.metamodel.FrameBound; +import org.finos.legend.engine.protocol.sql.metamodel.FunctionCall; +import org.finos.legend.engine.protocol.sql.metamodel.Group; +import org.finos.legend.engine.protocol.sql.metamodel.InListExpression; +import org.finos.legend.engine.protocol.sql.metamodel.InPredicate; +import org.finos.legend.engine.protocol.sql.metamodel.IntegerLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.IntervalLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.IsNotNullPredicate; +import org.finos.legend.engine.protocol.sql.metamodel.IsNullPredicate; +import org.finos.legend.engine.protocol.sql.metamodel.Join; +import org.finos.legend.engine.protocol.sql.metamodel.JoinOn; +import org.finos.legend.engine.protocol.sql.metamodel.LikePredicate; +import org.finos.legend.engine.protocol.sql.metamodel.Literal; +import org.finos.legend.engine.protocol.sql.metamodel.LogicalBinaryExpression; +import org.finos.legend.engine.protocol.sql.metamodel.LongLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.NamedArgumentExpression; +import org.finos.legend.engine.protocol.sql.metamodel.NegativeExpression; +import org.finos.legend.engine.protocol.sql.metamodel.Node; +import org.finos.legend.engine.protocol.sql.metamodel.NodeVisitor; +import org.finos.legend.engine.protocol.sql.metamodel.NotExpression; +import org.finos.legend.engine.protocol.sql.metamodel.NullLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.ParameterExpression; +import org.finos.legend.engine.protocol.sql.metamodel.QualifiedNameReference; +import org.finos.legend.engine.protocol.sql.metamodel.Query; +import org.finos.legend.engine.protocol.sql.metamodel.QueryBody; +import org.finos.legend.engine.protocol.sql.metamodel.QuerySpecification; +import org.finos.legend.engine.protocol.sql.metamodel.Relation; +import org.finos.legend.engine.protocol.sql.metamodel.SearchedCaseExpression; +import org.finos.legend.engine.protocol.sql.metamodel.Select; +import org.finos.legend.engine.protocol.sql.metamodel.SelectItem; +import org.finos.legend.engine.protocol.sql.metamodel.SetOperation; +import org.finos.legend.engine.protocol.sql.metamodel.SimpleCaseExpression; +import org.finos.legend.engine.protocol.sql.metamodel.SingleColumn; +import org.finos.legend.engine.protocol.sql.metamodel.SortItem; +import org.finos.legend.engine.protocol.sql.metamodel.Statement; +import org.finos.legend.engine.protocol.sql.metamodel.StringLiteral; +import org.finos.legend.engine.protocol.sql.metamodel.SubqueryExpression; +import org.finos.legend.engine.protocol.sql.metamodel.Table; +import org.finos.legend.engine.protocol.sql.metamodel.TableFunction; +import org.finos.legend.engine.protocol.sql.metamodel.TableSubquery; +import org.finos.legend.engine.protocol.sql.metamodel.Trim; +import org.finos.legend.engine.protocol.sql.metamodel.Union; +import org.finos.legend.engine.protocol.sql.metamodel.WhenClause; +import org.finos.legend.engine.protocol.sql.metamodel.Window; +import org.finos.legend.engine.protocol.sql.metamodel.WindowFrame; + +import java.util.List; +import java.util.function.Predicate; + +public class BaseNodeModifierVisitor implements NodeVisitor +{ + @Override + public Node visit(AliasedRelation val) + { + val.relation = _visit(val.relation); + + return val; + } + + @Override + public Node visit(AllColumns val) + { + return val; + } + + @Override + public Node visit(ArithmeticExpression val) + { + val.left = _visit(val.left); + val.right = _visit(val.right); + + return val; + } + + @Override + public Node visit(ArrayLiteral val) + { + val.values = _visit(val.values); + + return val; + } + + @Override + public Node visit(BetweenPredicate val) + { + val.value = _visit(val.value); + val.min = _visit(val.min); + val.max = _visit(val.max); + + return val; + } + + @Override + public Node visit(BooleanLiteral val) + { + return val; + } + + @Override + public Node visit(Cast val) + { + val.expression = _visit(val.expression); + + return val; + } + + @Override + public Node visit(ColumnType val) + { + return val; + } + + @Override + public Node visit(ComparisonExpression val) + { + val.left = _visit(val.left); + val.right = _visit(val.right); + + return val; + } + + @Override + public Node visit(CurrentTime val) + { + return val; + } + + @Override + public Node visit(DoubleLiteral val) + { + return val; + } + + @Override + public Node visit(Expression val) + { + return val; + } + + @Override + public Node visit(Extract val) + { + val.expression = _visit(val.expression); + + return val; + } + + @Override + public Node visit(FrameBound val) + { + val.value = _visit(val.value); + + return val; + } + + @Override + public Node visit(FunctionCall val) + { + val.arguments = _visit(val.arguments); + + return val; + } + + @Override + public Node visit(Group val) + { + return val; + } + + @Override + public Node visit(InListExpression val) + { + val.values = _visit(val.values); + + return val; + } + + @Override + public Node visit(InPredicate val) + { + val.value = _visit(val.value); + val.valueList = _visit(val.valueList); + + return val; + } + + @Override + public Node visit(IntegerLiteral val) + { + return val; + } + + @Override + public Node visit(IntervalLiteral val) + { + return val; + } + + @Override + public Node visit(IsNotNullPredicate val) + { + val.value = _visit(val.value); + + return val; + } + + @Override + public Node visit(IsNullPredicate val) + { + val.value = _visit(val.value); + + return val; + } + + @Override + public Node visit(Join val) + { + val.left = (Relation) val.left.accept(this); + val.right = (Relation) val.right.accept(this); + + if (val.criteria instanceof JoinOn) + { + ((JoinOn) val.criteria).expression = (Expression) ((JoinOn) val.criteria).expression.accept(this); + } + + return val; + } + + @Override + public Node visit(LikePredicate val) + { + val.value = _visit(val.value); + val.pattern = _visit(val.pattern); + + return val; + } + + @Override + public Node visit(Literal val) + { + return val; + } + + @Override + public Node visit(LogicalBinaryExpression val) + { + val.left = _visit(val.left); + val.right = _visit(val.right); + + return val; + } + + @Override + public Node visit(LongLiteral val) + { + return val; + } + + @Override + public Node visit(NamedArgumentExpression val) + { + val.expression = _visit(val.expression); + + return val; + } + + @Override + public Node visit(NegativeExpression val) + { + val.value = _visit(val.value); + + return val; + } + + @Override + public Node visit(NotExpression val) + { + val.value = _visit(val.value); + + return val; + } + + @Override + public Node visit(NullLiteral val) + { + return val; + } + + @Override + public Node visit(ParameterExpression val) + { + return val; + } + + @Override + public Node visit(QualifiedNameReference val) + { + return val; + } + + @Override + public Node visit(Query val) + { + val.queryBody = (QueryBody) val.queryBody.accept(this); + val.limit = _visit(val.limit); + val.offset = _visit(val.offset); + + return val; + } + + @Override + public Node visit(QueryBody val) + { + return val; + } + + @Override + public Node visit(QuerySpecification val) + { + val.from = _visit(val.from); + val.where = _visit(val.where); + val.groupBy = _visit(val.groupBy, e -> !(e instanceof IntegerLiteral)); + val.having = _visit(val.having); + val.select = (Select) val.select.accept(this); + val.limit = _visit(val.limit); + val.offset = _visit(val.offset); + + return val; + } + + @Override + public Node visit(Relation val) + { + return val; + } + + @Override + public Node visit(SearchedCaseExpression val) + { + val.defaultValue = _visit(val.defaultValue); + val.whenClauses = _visit(val.whenClauses); + + return val; + } + + @Override + public Node visit(Select val) + { + val.selectItems = _visit(val.selectItems); + + return val; + } + + @Override + public Node visit(SelectItem val) + { + return val; + } + + @Override + public Node visit(SetOperation val) + { + return val; + } + + @Override + public Node visit(SimpleCaseExpression val) + { + val.defaultValue = _visit(val.defaultValue); + val.operand = _visit(val.operand); + val.whenClauses = _visit(val.whenClauses); + + return val; + } + + @Override + public Node visit(SingleColumn val) + { + val.expression = _visit(val.expression); + + return val; + } + + @Override + public Node visit(SortItem val) + { + val.sortKey = _visit(val.sortKey); + + return val; + } + + @Override + public Node visit(Statement val) + { + return val; + } + + @Override + public Node visit(StringLiteral val) + { + return val; + } + + @Override + public Node visit(SubqueryExpression val) + { + val.query = _visit(val.query); + + return val; + } + + @Override + public Node visit(Table val) + { + return val; + } + + @Override + public Node visit(TableFunction val) + { + val.functionCall = _visit(val.functionCall); + + return val; + } + + @Override + public Node visit(TableSubquery val) + { + val.query = _visit(val.query); + + return val; + } + + @Override + public Node visit(Trim val) + { + val.value = _visit(val.value); + val.characters = _visit(val.characters); + + return val; + } + + @Override + public Node visit(Union val) + { + val.left = _visit(val.left); + val.right = _visit(val.right); + + return val; + } + + @Override + public Node visit(WhenClause val) + { + val.result = _visit(val.result); + val.operand = _visit(val.operand); + + return val; + } + + @Override + public Node visit(Window val) + { + return val; + } + + @Override + public Node visit(WindowFrame val) + { + return val; + } + + protected List _visit(List nodes) + { + return _visit(nodes, n -> true); + } + + protected List _visit(List nodes, Predicate filter) + { + return nodes != null ? ListIterate.collect(nodes, n -> filter.test(n) ? (T) n.accept(this) : n) : null; + } + + protected T _visit(T node) + { + return node != null ? (T) node.accept(this) : null; + } +} \ No newline at end of file diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/QueryRealiaser.java b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/QueryRealiaser.java new file mode 100644 index 00000000000..f277349336b --- /dev/null +++ b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/QueryRealiaser.java @@ -0,0 +1,181 @@ +// Copyright 2023 Goldman Sachs +// +// 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 org.finos.legend.engine.query.sql.api; + +import org.apache.commons.lang3.StringUtils; +import org.eclipse.collections.impl.list.mutable.FastList; +import org.eclipse.collections.impl.map.mutable.UnifiedMap; +import org.finos.legend.engine.protocol.sql.metamodel.AliasedRelation; +import org.finos.legend.engine.protocol.sql.metamodel.AllColumns; +import org.finos.legend.engine.protocol.sql.metamodel.Join; +import org.finos.legend.engine.protocol.sql.metamodel.Node; +import org.finos.legend.engine.protocol.sql.metamodel.QualifiedNameReference; +import org.finos.legend.engine.protocol.sql.metamodel.Query; +import org.finos.legend.engine.protocol.sql.metamodel.QuerySpecification; +import org.finos.legend.engine.protocol.sql.metamodel.Relation; +import org.finos.legend.engine.protocol.sql.metamodel.Table; +import org.finos.legend.engine.protocol.sql.metamodel.TableFunction; +import org.finos.legend.engine.protocol.sql.metamodel.TableSubquery; +import org.finos.legend.engine.protocol.sql.metamodel.Union; + +import java.util.Map; + +/** + * this re-aliaser does 2 things + * 1. adds an alias to all non aliased tables/tableFuncs. + * 2. ensures all aliases are unique throughout the query. + * we do not re-alias top level aliases as they must be unique anyway and in edge cases the alias can end up + * forming part of column names, so for clarity best to expose this to the user as such + */ + +/** + * TODO: move into pure code + */ +public class QueryRealiaser extends BaseNodeModifierVisitor +{ + private boolean root = true; + private AliasedRelation currentAlias; + private Map realiases = UnifiedMap.newMap(); + private int counter = 1; + + private QueryRealiaser() + { + + } + + public static Query realias(Query query) + { + return (Query) query.accept(new QueryRealiaser()); + } + + @Override + public Node visit(AliasedRelation val) + { + //we store the current state and the inner state becomes empty + currentAlias = val; + Map scopeRealiases = UnifiedMap.newMap(realiases); + boolean inRoot = root; + + //reset the state + root = false; + realiases = UnifiedMap.newMap(); + AliasedRelation result = (AliasedRelation) super.visit(val); + + //we always rename non root aliases. + String alias = inRoot ? val.alias : val.alias + '_' + counter++; + + //we add/set the new alias/realias + scopeRealiases.put(val.alias, alias); + result.alias = alias; + + //we restore the outer state + root = inRoot; + realiases = scopeRealiases; + currentAlias = val; + return result; + } + + @Override + public Node visit(QualifiedNameReference val) + { + QualifiedNameReference result = (QualifiedNameReference) super.visit(val); + + if (result.name.parts.size() == 2) + { + result.name.parts = FastList.newListWith(realiases.get(result.name.parts.get(0)), result.name.parts.get(1)); + } + + return result; + } + + @Override + public Node visit(AllColumns val) + { + val.prefix = val.prefix != null ? realiases.get(val.prefix) : null; + + return val; + } + + @Override + public Node visit(Join val) + { + currentAlias = null; + return super.visit(val); + } + + @Override + public Node visit(QuerySpecification val) + { + currentAlias = null; + return super.visit(val); + } + + @Override + public Node visit(Table val) + { + return aliasIfRequired(val, StringUtils.join(val.name.parts, ".")); + } + + @Override + public Node visit(TableFunction val) + { + return aliasIfRequired(val, StringUtils.join(val.functionCall.name.parts, ".")); + } + + @Override + public Node visit(TableSubquery val) + { + //store the current state + boolean inRoot = root; + Map nodes = UnifiedMap.newMap(realiases); + + //reset the state for recurse + root = false; + currentAlias = null; + realiases = UnifiedMap.newMap(); + Node result = super.visit(val); + + //restore the state + realiases = nodes; + root = inRoot; + + return result; + } + + @Override + public Node visit(Union val) + { + currentAlias = null; + return super.visit(val); + } + + + private Node aliasIfRequired(Relation relation, String name) + { + if (currentAlias == null) + { + AliasedRelation a = new AliasedRelation(); + a.alias = "t" + counter++; + a.relation = relation; + + realiases.put(name, a.alias); + + return a; + } + + return relation; + } +} diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/SQLExecutor.java b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/SQLExecutor.java index 00f80b9a45a..bcb9f571a39 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/SQLExecutor.java +++ b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/main/java/org/finos/legend/engine/query/sql/api/SQLExecutor.java @@ -205,7 +205,9 @@ private T process(Query query, PureModel pureModel = modelManager.loadModel(pureModelContext, PureClientVersions.production, profiles, ""); - Root_meta_external_query_sql_metamodel_Query compiledQuery = new ProtocolToMetamodelTranslator().translate(query, pureModel); + Query realised = QueryRealiaser.realias(query); + + Root_meta_external_query_sql_metamodel_Query compiledQuery = new ProtocolToMetamodelTranslator().translate(realised, pureModel); RichIterable compiledSources = new SQLSourceTranslator().translate(sources, pureModel); LOGGER.info("{}", new LogInfo(profiles, LoggingEventType.GENERATE_PLAN_START)); diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/TestQueryRealiaser.java b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/TestQueryRealiaser.java new file mode 100644 index 00000000000..8a108da72e0 --- /dev/null +++ b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/TestQueryRealiaser.java @@ -0,0 +1,89 @@ +// Copyright 2023 Goldman Sachs +// +// 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 org.finos.legend.engine.query.sql.api; + +import org.finos.legend.engine.language.sql.grammar.from.SQLGrammarParser; +import org.finos.legend.engine.language.sql.grammar.to.SQLGrammarComposer; +import org.finos.legend.engine.protocol.sql.metamodel.Query; +import org.junit.Assert; +import org.junit.Test; + +public class TestQueryRealiaser +{ + + @Test + public void testSelect() + { + test("select a from myTable as t1"); + + test("select t1.a from myTable as t1"); + + test("select myTable.a from myTable", + "select t1.a from myTable as t1"); + + test("select a from myTable", + "select a from myTable as t1"); + } + + @Test + public void testUnion() + { + test("select a from myTable as t1 union select a from myTable as t2"); + test("select a from myTable as t1 union select a from myTable as t2"); + test("select a from myTable union select a from myTable", "select a from myTable as t1 union select a from myTable as t2"); + + test("select * from (select a from myTable as t1 union select a from myTable as t2) t1", + "select * from (select a from myTable as t1_1 union select a from myTable as t2_2) as t1"); + + test("select * from (select t1.a from myTable as t1 union select t2.a from myTable as t2) t1", + "select * from (select t1_1.a from myTable as t1_1 union select t2_2.a from myTable as t2_2) as t1"); + } + + @Test + public void testJoin() + { + test("select t1.a, t2.b from myTable as t1 left outer join myTable2 as t2 on (t1.a = t2.b)"); + + test("select t1.a, t2.b from myTable as t1 left outer join (select t1.* from myTable as t1) as t2 on (t1.a = t2.b)", + "select t1.a, t2.b from myTable as t1 left outer join (select t1_1.* from myTable as t1_1) as t2 on (t1.a = t2.b)"); + } + + @Test + public void testSubQuery() + { + test("select * from (select a from (select * from myTable left outer join myTable2 on myTable.a = myTable2.b))", + "select * from (select a from (select * from myTable as t1 left outer join myTable2 as t2 on (t1.a = t2.b)))"); + + test("select * from (select a from (select * from myTable left outer join myTable2 on myTable.a = myTable2.b) as myTable) as myTable", + "select * from (select a from (select * from myTable as t1 left outer join myTable2 as t2 on (t1.a = t2.b)) as myTable_3) as myTable"); + } + + private void test(String input) + { + test(input, input); + } + + private void test(String input, String expected) + { + Query query = (Query) SQLGrammarParser.newInstance().parseStatement(input); + + Query output = QueryRealiaser.realias(query); + + String grammar = SQLGrammarComposer.newInstance().renderNode(output); + + Assert.assertEquals(expected.toLowerCase(), grammar.toLowerCase()); + } +} \ No newline at end of file diff --git a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/execute/SqlExecuteTest.java b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/execute/SqlExecuteTest.java index c2f383ee6e5..183bc2296ce 100644 --- a/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/execute/SqlExecuteTest.java +++ b/legend-engine-xts-sql/legend-engine-xt-sql-query/src/test/java/org/finos/legend/engine/query/sql/api/execute/SqlExecuteTest.java @@ -186,6 +186,27 @@ public void testExecuteWithExpressionParams() throws JsonProcessingException Assert.assertEquals(allExpected, OM.readValue(all, TDSExecuteResult.class)); } + @Test + public void testExecuteMultiUnionNoAliases() throws JsonProcessingException + { + //this is to test the query realiasing, can be moved to testTranspile one realiser moved to pure code + String all = resources.target("sql/v1/execution/executeQueryString") + .request() + .post(Entity.text("SELECT Name FROM service('/personServiceForStartDate/{date}', date =>'2023-08-24') " + + "UNION SELECT Name FROM service('/personServiceForStartDate/{date}', date =>'2023-08-24') " + + "UNION SELECT Name FROM service('/personServiceForStartDate/{date}', date =>'2023-08-24') " + + "UNION SELECT Name FROM service('/personServiceForStartDate/{date}', date =>'2023-08-24')")).readEntity(String.class); + + TDSExecuteResult allExpected = TDSExecuteResult.builder(FastList.newListWith("Name")) + .addRow(FastList.newListWith("Alice")) + .addRow(FastList.newListWith("Alice")) + .addRow(FastList.newListWith("Alice")) + .addRow(FastList.newListWith("Alice")) + .build(); + + Assert.assertEquals(allExpected, OM.readValue(all, TDSExecuteResult.class)); + } + @Test public void testExecuteWithCSVFormat() {