diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 499f27f00e7e0..f40412f0a6ee9 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -548,8 +548,7 @@ valueExpression ; primaryExpression - : name=(CURRENT_DATE | CURRENT_TIMESTAMP) #timeFunctionCall - | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase + : CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct @@ -727,7 +726,7 @@ nonReserved | NULL | ORDER | OUTER | TABLE | TRUE | WITH | RLIKE | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN | UNBOUNDED | WHEN - | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT | CURRENT_DATE | CURRENT_TIMESTAMP + | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT ; SELECT: 'SELECT'; @@ -955,8 +954,6 @@ OPTION: 'OPTION'; ANTI: 'ANTI'; LOCAL: 'LOCAL'; INPATH: 'INPATH'; -CURRENT_DATE: 'CURRENT_DATE'; -CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; STRING : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index f707aa820ee57..8e2ece4995b56 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -854,7 +854,12 @@ class Analyzer( q.transformExpressionsUp { case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } + val result = + withPosition(u) { + q.resolveChildren(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, q)) + .getOrElse(u) + } logDebug(s"Resolving $u to $result") result case UnresolvedExtractValue(child, fieldExpr) if child.resolved => @@ -932,6 +937,30 @@ class Analyzer( exprs.exists(_.find(_.isInstanceOf[UnresolvedDeserializer]).isDefined) } + /** + * Literal functions do not require the user to specify braces when calling them + * When an attributes is not resolvable, we try to resolve it as a literal function. + */ + private def resolveLiteralFunction( + nameParts: Seq[String], + attribute: UnresolvedAttribute, + plan: LogicalPlan): Option[Expression] = { + if (nameParts.length != 1) return None + val isNamedExpression = plan match { + case Aggregate(_, aggregateExpressions, _) => aggregateExpressions.contains(attribute) + case Project(projectList, _) => projectList.contains(attribute) + case Window(windowExpressions, _, _, _) => windowExpressions.contains(attribute) + case _ => false + } + val wrapper: Expression => Expression = + if (isNamedExpression) f => Alias(f, toPrettySQL(f))() else identity + // support CURRENT_DATE and CURRENT_TIMESTAMP + val literalFunctions = Seq(CurrentDate(), CurrentTimestamp()) + val name = nameParts.head + val func = literalFunctions.find(e => resolver(e.prettyName, name)) + func.map(wrapper) + } + protected[sql] def resolveExpression( expr: Expression, plan: LogicalPlan, @@ -944,7 +973,11 @@ class Analyzer( expr transformUp { case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) case u @ UnresolvedAttribute(nameParts) => - withPosition(u) { plan.resolve(nameParts, resolver).getOrElse(u) } + withPosition(u) { + plan.resolve(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, plan)) + .getOrElse(u) + } case UnresolvedExtractValue(child, fieldName) if child.resolved => ExtractValue(child, fieldName, resolver) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index d1c9332bee18b..c15899cb230e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1083,19 +1083,6 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } - /** - * Create a current timestamp/date expression. These are different from regular function because - * they do not require the user to specify braces when calling them. - */ - override def visitTimeFunctionCall(ctx: TimeFunctionCallContext): Expression = withOrigin(ctx) { - ctx.name.getType match { - case SqlBaseParser.CURRENT_DATE => - CurrentDate() - case SqlBaseParser.CURRENT_TIMESTAMP => - CurrentTimestamp() - } - } - /** * Create a function database (optional) and name pair. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index f06219198bb58..5ae8d2e849828 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -590,11 +590,6 @@ class ExpressionParserSuite extends PlanTest { intercept("1 - f('o', o(bar)) hello * world", "mismatched input '*'") } - test("current date/timestamp braceless expressions") { - assertEqual("current_date", CurrentDate()) - assertEqual("current_timestamp", CurrentTimestamp()) - } - test("SPARK-17364, fully qualified column name which starts with number") { assertEqual("123_", UnresolvedAttribute("123_")) assertEqual("1a.123_", UnresolvedAttribute("1a.123_")) diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 3fd1c37e71795..b42e92436da43 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -2,3 +2,20 @@ -- [SPARK-16836] current_date and current_timestamp literals select current_date = current_date(), current_timestamp = current_timestamp(); + +-- [SPARK-22333]: timeFunctionCall has conflicts with columnReference +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(current_date, current_timestamp); + +select current_date, current_timestamp from ttf1; + +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b); + +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2; + +select a, b from ttf2 order by a, current_date; diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 032e4258500fb..2d15718d3f8ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 1 +-- Number of queries: 6 -- !query 0 @@ -8,3 +8,50 @@ select current_date = current_date(), current_timestamp = current_timestamp() struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean> -- !query 0 output true true + + +-- !query 1 +create temporary view ttf1 as select * from values + (1, 2), + (2, 3) + as ttf1(current_date, current_timestamp) +-- !query 1 schema +struct<> +-- !query 1 output + + +-- !query 2 +select current_date, current_timestamp from ttf1 +-- !query 2 schema +struct +-- !query 2 output +1 2 +2 3 + + +-- !query 3 +create temporary view ttf2 as select * from values + (1, 2), + (2, 3) + as ttf2(a, b) +-- !query 3 schema +struct<> +-- !query 3 output + + +-- !query 4 +select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2 +-- !query 4 schema +struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int> +-- !query 4 output +true true 1 2 +true true 2 3 + + +-- !query 5 +select a, b from ttf2 order by a, current_date +-- !query 5 schema +struct +-- !query 5 output +1 2 +2 3