Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -564,8 +564,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
Expand Down Expand Up @@ -747,7 +746,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
| DIRECTORY
| BOTH | LEADING | TRAILING
;
Expand Down Expand Up @@ -983,8 +982,6 @@ OPTION: 'OPTION';
ANTI: 'ANTI';
LOCAL: 'LOCAL';
INPATH: 'INPATH';
CURRENT_DATE: 'CURRENT_DATE';
CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP';

STRING
: '\'' ( ~('\''|'\\') | ('\\' .) )* '\''
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -786,7 +786,12 @@ class Analyzer(
private def resolve(e: Expression, q: LogicalPlan): Expression = e match {
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 =>
Expand Down Expand Up @@ -925,6 +930,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,
Expand All @@ -937,7 +966,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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1234,19 +1234,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.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -592,11 +592,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_"))
Expand Down
17 changes: 17 additions & 0 deletions sql/core/src/test/resources/sql-tests/inputs/datetime.sql
Original file line number Diff line number Diff line change
Expand Up @@ -8,3 +8,20 @@ select to_date(null), to_date('2016-12-31'), to_date('2016-12-31', 'yyyy-MM-dd')
select to_timestamp(null), to_timestamp('2016-12-31 00:12:00'), to_timestamp('2016-12-31', 'yyyy-MM-dd');

select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27'), dayofweek(null), dayofweek('1582-10-15 13:10:15');

-- [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;
49 changes: 48 additions & 1 deletion sql/core/src/test/resources/sql-tests/results/datetime.sql.out
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
-- Number of queries: 4
-- Number of queries: 9


-- !query 0
Expand Down Expand Up @@ -32,3 +32,50 @@ select dayofweek('2007-02-03'), dayofweek('2009-07-30'), dayofweek('2017-05-27')
struct<dayofweek(CAST(2007-02-03 AS DATE)):int,dayofweek(CAST(2009-07-30 AS DATE)):int,dayofweek(CAST(2017-05-27 AS DATE)):int,dayofweek(CAST(NULL AS DATE)):int,dayofweek(CAST(1582-10-15 13:10:15 AS DATE)):int>
-- !query 3 output
7 5 7 NULL 6


-- !query 4
create temporary view ttf1 as select * from values
(1, 2),
(2, 3)
as ttf1(current_date, current_timestamp)
-- !query 4 schema
struct<>
-- !query 4 output


-- !query 5
select current_date, current_timestamp from ttf1
-- !query 5 schema
struct<current_date:int,current_timestamp:int>
-- !query 5 output
1 2
2 3


-- !query 6
create temporary view ttf2 as select * from values
(1, 2),
(2, 3)
as ttf2(a, b)
-- !query 6 schema
struct<>
-- !query 6 output


-- !query 7
select current_date = current_date(), current_timestamp = current_timestamp(), a, b from ttf2
-- !query 7 schema
struct<(current_date() = current_date()):boolean,(current_timestamp() = current_timestamp()):boolean,a:int,b:int>
-- !query 7 output
true true 1 2
true true 2 3


-- !query 8
select a, b from ttf2 order by a, current_date
-- !query 8 schema
struct<a:int,b:int>
-- !query 8 output
1 2
2 3