Skip to content

Commit 36b4bbb

Browse files
committed
support literal function in resolveExpression
1 parent a96d945 commit 36b4bbb

File tree

1 file changed

+30
-26
lines changed
  • sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis

1 file changed

+30
-26
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala

Lines changed: 30 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -783,30 +783,6 @@ class Analyzer(
783783
}
784784
}
785785

786-
/**
787-
* Literal functions do not require the user to specify braces when calling them
788-
* When an attributes is not resolvable, we try to resolve it as a literal function.
789-
*/
790-
private def resolveAsLiteralFunctions(
791-
nameParts: Seq[String],
792-
attribute: UnresolvedAttribute,
793-
plan: LogicalPlan): Option[Expression] = {
794-
if (nameParts.length != 1) return None
795-
val isNamedExpression = plan match {
796-
case Aggregate(_, aggregateExpressions, _) => aggregateExpressions.contains(attribute)
797-
case Project(projectList, _) => projectList.contains(attribute)
798-
case Window(windowExpressions, _, _, _) => windowExpressions.contains(attribute)
799-
case _ => false
800-
}
801-
val wrapper: Expression => Expression =
802-
if (isNamedExpression) f => Alias(f, toPrettySQL(f))() else identity
803-
// support CURRENT_DATE and CURRENT_TIMESTAMP
804-
val literalFunctions = Seq(CurrentDate(), CurrentTimestamp())
805-
val name = nameParts.head
806-
val func = literalFunctions.find(e => resolver(e.prettyName, name))
807-
func.map(wrapper)
808-
}
809-
810786
def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators {
811787
case p: LogicalPlan if !p.childrenResolved => p
812788

@@ -871,7 +847,7 @@ class Analyzer(
871847
val result =
872848
withPosition(u) {
873849
q.resolveChildren(nameParts, resolver)
874-
.orElse(resolveAsLiteralFunctions(nameParts, u, q))
850+
.orElse(resolveLiteralFunction(nameParts, u, q))
875851
.getOrElse(u)
876852
}
877853
logDebug(s"Resolving $u to $result")
@@ -951,6 +927,30 @@ class Analyzer(
951927
exprs.exists(_.find(_.isInstanceOf[UnresolvedDeserializer]).isDefined)
952928
}
953929

930+
/**
931+
* Literal functions do not require the user to specify braces when calling them
932+
* When an attributes is not resolvable, we try to resolve it as a literal function.
933+
*/
934+
private def resolveLiteralFunction(
935+
nameParts: Seq[String],
936+
attribute: UnresolvedAttribute,
937+
plan: LogicalPlan): Option[Expression] = {
938+
if (nameParts.length != 1) return None
939+
val isNamedExpression = plan match {
940+
case Aggregate(_, aggregateExpressions, _) => aggregateExpressions.contains(attribute)
941+
case Project(projectList, _) => projectList.contains(attribute)
942+
case Window(windowExpressions, _, _, _) => windowExpressions.contains(attribute)
943+
case _ => false
944+
}
945+
val wrapper: Expression => Expression =
946+
if (isNamedExpression) f => Alias(f, toPrettySQL(f))() else identity
947+
// support CURRENT_DATE and CURRENT_TIMESTAMP
948+
val literalFunctions = Seq(CurrentDate(), CurrentTimestamp())
949+
val name = nameParts.head
950+
val func = literalFunctions.find(e => resolver(e.prettyName, name))
951+
func.map(wrapper)
952+
}
953+
954954
protected[sql] def resolveExpression(
955955
expr: Expression,
956956
plan: LogicalPlan,
@@ -963,7 +963,11 @@ class Analyzer(
963963
expr transformUp {
964964
case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal)
965965
case u @ UnresolvedAttribute(nameParts) =>
966-
withPosition(u) { plan.resolve(nameParts, resolver).getOrElse(u) }
966+
withPosition(u) {
967+
plan.resolveChildren(nameParts, resolver)
968+
.orElse(resolveLiteralFunction(nameParts, u, plan))
969+
.getOrElse(u)
970+
}
967971
case UnresolvedExtractValue(child, fieldName) if child.resolved =>
968972
ExtractValue(child, fieldName, resolver)
969973
}

0 commit comments

Comments
 (0)