Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
89 changes: 47 additions & 42 deletions core/trino-grammar/src/main/antlr4/io/trino/grammar/sql/SqlBase.g4
Original file line number Diff line number Diff line change
Expand Up @@ -586,66 +586,70 @@ valueExpression
;

primaryExpression
: literal #literals
| QUESTION_MARK #parameter
| POSITION '(' valueExpression IN valueExpression ')' #position
| '(' expression (',' expression)+ ')' #rowConstructor
| ROW '(' expression (',' expression)* ')' #rowConstructor
: literal #literals
| QUESTION_MARK #parameter
| POSITION '(' valueExpression IN valueExpression ')' #position
| '(' expression (',' expression)+ ')' #rowConstructor
| ROW '(' expression (',' expression)* ')' #rowConstructor
| name=LISTAGG '(' setQuantifier? expression (',' string)?
(ON OVERFLOW listAggOverflowBehavior)? ')'
(WITHIN GROUP '(' orderBy ')')
filter? over? #listagg
filter? over? #listagg
| processingMode? qualifiedName '(' (label=identifier '.')? ASTERISK ')'
filter? over? #functionCall
filter? over? #functionCall
| processingMode? qualifiedName '(' (setQuantifier? expression (',' expression)*)?
orderBy? ')' filter? (nullTreatment? over)? #functionCall
| identifier over #measure
| identifier '->' expression #lambda
| '(' (identifier (',' identifier)*)? ')' '->' expression #lambda
| '(' query ')' #subqueryExpression
orderBy? ')' filter? (nullTreatment? over)? #functionCall
| identifier over #measure
| identifier '->' expression #lambda
| '(' (identifier (',' identifier)*)? ')' '->' expression #lambda
| '(' query ')' #subqueryExpression
// This is an extension to ANSI SQL, which considers EXISTS to be a <boolean expression>
| EXISTS '(' query ')' #exists
| CASE operand=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase
| CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase
| CAST '(' expression AS type ')' #cast
| TRY_CAST '(' expression AS type ')' #cast
| ARRAY '[' (expression (',' expression)*)? ']' #arrayConstructor
| '[' (expression (',' expression)*)? ']' #arrayConstructor
| value=primaryExpression '[' index=valueExpression ']' #subscript
| identifier #columnReference
| base=primaryExpression '.' fieldName=identifier #dereference
| name=CURRENT_DATE #currentDate
| name=CURRENT_TIME ('(' precision=INTEGER_VALUE ')')? #currentTime
| name=CURRENT_TIMESTAMP ('(' precision=INTEGER_VALUE ')')? #currentTimestamp
| name=LOCALTIME ('(' precision=INTEGER_VALUE ')')? #localTime
| name=LOCALTIMESTAMP ('(' precision=INTEGER_VALUE ')')? #localTimestamp
| name=CURRENT_USER #currentUser
| name=CURRENT_CATALOG #currentCatalog
| name=CURRENT_SCHEMA #currentSchema
| name=CURRENT_PATH #currentPath
| TRIM '(' (trimsSpecification? trimChar=valueExpression? FROM)?
trimSource=valueExpression ')' #trim
| TRIM '(' trimSource=valueExpression ',' trimChar=valueExpression ')' #trim
| SUBSTRING '(' valueExpression FROM valueExpression (FOR valueExpression)? ')' #substring
| NORMALIZE '(' valueExpression (',' normalForm)? ')' #normalize
| EXTRACT '(' identifier FROM valueExpression ')' #extract
| '(' expression ')' #parenthesizedExpression
| GROUPING '(' (qualifiedName (',' qualifiedName)*)? ')' #groupingOperation
| JSON_EXISTS '(' jsonPathInvocation (jsonExistsErrorBehavior ON ERROR)? ')' #jsonExists
| EXISTS '(' query ')' #exists
| CASE operand=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase
| CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase
| CAST '(' expression AS type ')' #cast
| TRY_CAST '(' expression AS type ')' #cast
// the target is a primaryExpression to support PostgreSQL-style casts
// of the form <complex expression>::<type>, which are syntactically ambiguous with
// static method calls defined by the SQL spec (and we reserve it for future use)
| primaryExpression DOUBLE_COLON identifier ('(' (expression (',' expression)*)? ')')? #staticMethodCall
Copy link
Member

Choose a reason for hiding this comment

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

use type on the rhs?

AFAIR engines allow a::timestamp with time zone and likely a::double precision too

Copy link
Member Author

Choose a reason for hiding this comment

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

Can’t use type, and hence the limitation I mentioned in the description. The standard SQL syntax requires this to be a function invocation, so it has to look like it.

Copy link
Member

Choose a reason for hiding this comment

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

to me, "syntactically ambiguous with static method calls" means that foo::bar(1) is ambiguous in a sense that it can be two different things

  • PostgreSQL-style cast of expression foo to type bar(1)
  • a static method bar invocation on type foo with parameters 1

it is surprising, but it seems to me that foo::bar is also ambiguous because argument braces are optional

image

now,
foo::timestamp already departs from the spec
foo::timestamp with time zone also departs from the spec, but is consistent with foo::timestamp. And is not ambiguous.

Copy link
Member Author

Choose a reason for hiding this comment

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

"syntactically ambiguous with static method calls" means that foo::bar(1) is ambiguous in a sense that it can be two different things

Exactly. That's a syntactic ambiguity that's easy to solve during analysis. We parse all occurrences of :: as static method calls and decide later if it was meant to be a call or a cast depending on what's on the left side of the expression. If it's a type-producing expression (currently, an identifier matching a type name), then we treat it as a static method call. Otherwise, we treat it as a cast.

foo::timestamp with time zone also departs from the spec, but is consistent with foo::timestamp. And is not ambiguous.

Right, but to be able to parse that we'd need to allow either identifier ('(' (expression (',' expression)*)? ')') or type on the right side, which introduces all sorts of problems. See my comment below.

Also, as an aside, note that there's another ambiguous usage (I mention it in the comment below, too) with:

SELECT xxx::double precision

In Trino, that could be interpreted either as SELECT xxx::double AS precision or as a cast to double precision. PostgreSQL doesn't have that problem since double is not a valid type. Which goes to say, sometimes we can't just adopt a language feature from another database, since there are intricate interactions between all features that need to be taken into account.

| ARRAY '[' (expression (',' expression)*)? ']' #arrayConstructor
| '[' (expression (',' expression)*)? ']' #arrayConstructor
| value=primaryExpression '[' index=valueExpression ']' #subscript
| identifier #columnReference
| base=primaryExpression '.' fieldName=identifier #dereference
| name=CURRENT_DATE #currentDate
| name=CURRENT_TIME ('(' precision=INTEGER_VALUE ')')? #currentTime
| name=CURRENT_TIMESTAMP ('(' precision=INTEGER_VALUE ')')? #currentTimestamp
| name=LOCALTIME ('(' precision=INTEGER_VALUE ')')? #localTime
| name=LOCALTIMESTAMP ('(' precision=INTEGER_VALUE ')')? #localTimestamp
| name=CURRENT_USER #currentUser
| name=CURRENT_CATALOG #currentCatalog
| name=CURRENT_SCHEMA #currentSchema
| name=CURRENT_PATH #currentPath
| TRIM '(' (trimsSpecification? trimChar=valueExpression? FROM)?
trimSource=valueExpression ')' #trim
| TRIM '(' trimSource=valueExpression ',' trimChar=valueExpression ')' #trim
| SUBSTRING '(' valueExpression FROM valueExpression (FOR valueExpression)? ')' #substring
| NORMALIZE '(' valueExpression (',' normalForm)? ')' #normalize
| EXTRACT '(' identifier FROM valueExpression ')' #extract
| '(' expression ')' #parenthesizedExpression
| GROUPING '(' (qualifiedName (',' qualifiedName)*)? ')' #groupingOperation
| JSON_EXISTS '(' jsonPathInvocation (jsonExistsErrorBehavior ON ERROR)? ')' #jsonExists
| JSON_VALUE '('
jsonPathInvocation
(RETURNING type)?
(emptyBehavior=jsonValueBehavior ON EMPTY)?
(errorBehavior=jsonValueBehavior ON ERROR)?
')' #jsonValue
')' #jsonValue
| JSON_QUERY '('
jsonPathInvocation
(RETURNING type (FORMAT jsonRepresentation)?)?
(jsonQueryWrapperBehavior WRAPPER)?
((KEEP | OMIT) QUOTES (ON SCALAR TEXT_STRING)?)?
(emptyBehavior=jsonQueryBehavior ON EMPTY)?
(errorBehavior=jsonQueryBehavior ON ERROR)?
')' #jsonQuery
')' #jsonQuery
| JSON_OBJECT '('
(
jsonObjectMember (',' jsonObjectMember)*
Expand Down Expand Up @@ -1123,6 +1127,7 @@ DISTINCT: 'DISTINCT';
DISTRIBUTED: 'DISTRIBUTED';
DO: 'DO';
DOUBLE: 'DOUBLE';
DOUBLE_COLON: '::';
DROP: 'DROP';
ELSE: 'ELSE';
EMPTY: 'EMPTY';
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,7 @@
import io.trino.sql.tree.SkipTo;
import io.trino.sql.tree.SortItem;
import io.trino.sql.tree.SortItem.Ordering;
import io.trino.sql.tree.StaticMethodCall;
import io.trino.sql.tree.StringLiteral;
import io.trino.sql.tree.SubqueryExpression;
import io.trino.sql.tree.SubscriptExpression;
Expand Down Expand Up @@ -1704,6 +1705,43 @@ private void analyzeFrameRangeOffset(Expression offsetValue, FrameBound.Type bou
frameBoundCalculations.put(NodeRef.of(offsetValue), function);
}

@Override
protected Type visitStaticMethodCall(StaticMethodCall node, Context context)
{
// PostgreSQL-style casts are syntactically ambiguous with static method calls. So, static method call semantics take precendence.
// A static method call is characterized by the target being an expression whose type is "type". This not yet supported
// as a first-class concept, so we fake it by analyzing the expression normally. If the analysis succeeds, we treat it as
// the target of a cast.

// Trino allows resolving column names that match type names, so we need to check explicitly
// if this is a type reference in the context of a static method call
if (node.getTarget() instanceof Identifier target) {
try {
plannerContext.getTypeManager().fromSqlType(target.getValue());
throw semanticException(NOT_SUPPORTED, node, "Static method calls are not supported");
}
catch (TypeNotFoundException typeException) {
// since the type is not found, this must be a normal value-producing expression. Treat it as a candidate for
// resolving the PostgreSQL-style cast, as explained above.
}
}

if (!node.getArguments().isEmpty()) {
throw semanticException(NOT_SUPPORTED, node, "Static method calls are not supported");
}

process(node.getTarget(), context);

// assume it's a PostgreSQL-style cast unless result type is not a known type
try {
Type type = plannerContext.getTypeManager().fromSqlType(node.getMethod().getValue());
return setExpressionType(node, type);
}
catch (Exception e) {
throw semanticException(NOT_SUPPORTED, node, "Static method calls are not supported");
}
}

@Override
protected Type visitWindowOperation(WindowOperation node, Context context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@
import io.trino.sql.tree.Row;
import io.trino.sql.tree.SearchedCaseExpression;
import io.trino.sql.tree.SimpleCaseExpression;
import io.trino.sql.tree.StaticMethodCall;
import io.trino.sql.tree.StringLiteral;
import io.trino.sql.tree.SubscriptExpression;
import io.trino.sql.tree.Trim;
Expand Down Expand Up @@ -316,6 +317,7 @@ private io.trino.sql.ir.Expression translate(Expression expr, boolean isRoot)
case io.trino.sql.tree.FieldReference expression -> translate(expression);
case Identifier expression -> translate(expression);
case FunctionCall expression -> translate(expression);
case StaticMethodCall expression -> translate(expression);
case DereferenceExpression expression -> translate(expression);
case Array expression -> translate(expression);
case CurrentCatalog expression -> translate(expression);
Expand Down Expand Up @@ -663,6 +665,14 @@ private io.trino.sql.ir.Expression translate(FunctionCall expression)
.collect(toImmutableList()));
}

private io.trino.sql.ir.Expression translate(StaticMethodCall expression)
{
// Currently, only PostgreSQL-style cast shorthand expressions are supported
return new io.trino.sql.ir.Cast(
translateExpression(expression.getTarget()),
analysis.getType(expression));
}

private io.trino.sql.ir.Expression translate(DereferenceExpression expression)
{
if (analysis.isColumnReference(expression)) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.trino.operator.scalar;

import io.trino.spi.type.DoubleType;
import io.trino.spi.type.VarcharType;
import io.trino.sql.query.QueryAssertions;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import org.junit.jupiter.api.parallel.Execution;

import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT;

@TestInstance(PER_CLASS)
@Execution(CONCURRENT)
public class TestStaticMethodCall
{
private QueryAssertions assertions;

@BeforeAll
public void init()
{
assertions = new QueryAssertions();
}

@AfterAll
public void teardown()
{
assertions.close();
assertions = null;
}

@Test
void testPostgreSqlStyleCast()
{
assertThat(assertions.expression("1::double"))
.hasType(DoubleType.DOUBLE)
.isEqualTo(1.0);

assertThat(assertions.expression("1::varchar"))
.hasType(VarcharType.VARCHAR)
.isEqualTo("1");

assertThatThrownBy(() -> assertions.expression("1::varchar(100)").evaluate())
.hasMessage("line 1:13: Static method calls are not supported");

assertThat(assertions.expression("(a + b)::double")
.binding("a", "1")
.binding("b", "2"))
.hasType(DoubleType.DOUBLE)
.isEqualTo(3.0);

assertThatThrownBy(() -> assertions.expression("1::decimal(3, 2)").evaluate())
.hasMessage("line 1:13: Static method calls are not supported");
}

@Test
void testCall()
{
assertThatThrownBy(() -> assertions.expression("1::double(2)").evaluate())
.hasMessage("line 1:13: Static method calls are not supported");

assertThatThrownBy(() -> assertions.expression("1::foo").evaluate())
.hasMessage("line 1:13: Static method calls are not supported");

assertThatThrownBy(() -> assertions.expression("integer::foo").evaluate())
.hasMessage("line 1:19: Static method calls are not supported");

assertThatThrownBy(() -> assertions.expression("integer::foo(1, 2)").evaluate())
.hasMessage("line 1:19: Static method calls are not supported");

assertThat(assertions.query("SELECT bigint::real FROM (VALUES 1) AS t(bigint)"))
.failure()
.hasMessage("line 1:14: Static method calls are not supported");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@
import io.trino.sql.tree.SimpleGroupBy;
import io.trino.sql.tree.SkipTo;
import io.trino.sql.tree.SortItem;
import io.trino.sql.tree.StaticMethodCall;
import io.trino.sql.tree.StringLiteral;
import io.trino.sql.tree.SubqueryExpression;
import io.trino.sql.tree.SubscriptExpression;
Expand Down Expand Up @@ -467,6 +468,24 @@ protected String visitFunctionCall(FunctionCall node, Void context)
return builder.toString();
}

@Override
protected String visitStaticMethodCall(StaticMethodCall node, Void context)
{
StringBuilder builder = new StringBuilder();

builder.append(process(node.getTarget(), context))
.append("::")
.append(process(node.getMethod(), context));

if (!node.getArguments().isEmpty()) {
builder.append('(')
.append(joinExpressions(node.getArguments()))
.append(')');
}

return builder.toString();
}

@Override
protected String visitWindowOperation(WindowOperation node, Void context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,7 @@
import io.trino.sql.tree.SortItem;
import io.trino.sql.tree.StartTransaction;
import io.trino.sql.tree.Statement;
import io.trino.sql.tree.StaticMethodCall;
import io.trino.sql.tree.StringLiteral;
import io.trino.sql.tree.SubqueryExpression;
import io.trino.sql.tree.SubscriptExpression;
Expand Down Expand Up @@ -3104,6 +3105,16 @@ else if (processingMode.FINAL() != null) {
arguments);
}

@Override
public Node visitStaticMethodCall(SqlBaseParser.StaticMethodCallContext context)
{
return new StaticMethodCall(
getLocation(context.DOUBLE_COLON()),
(Expression) visit(context.primaryExpression()),
(Identifier) visit(context.identifier()),
visit(context.expression(), Expression.class));
}

@Override
public Node visitMeasure(SqlBaseParser.MeasureContext context)
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -322,6 +322,11 @@ protected R visitFunctionCall(FunctionCall node, C context)
return visitExpression(node, context);
}

protected R visitStaticMethodCall(StaticMethodCall node, C context)
{
return visitExpression(node, context);
}

protected R visitProcessingMode(ProcessingMode node, C context)
{
return visitNode(node, context);
Expand Down
Loading