You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/02/10 05:11:14 UTC

[GitHub] [spark] cloud-fan commented on a change in pull request #35404: [SPARK-38118][SQL] Func(wrong data type) in the HAVING claus should throw data mismatch error

cloud-fan commented on a change in pull request #35404:
URL: https://github.com/apache/spark/pull/35404#discussion_r803306323



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
##########
@@ -4249,7 +4250,30 @@ object ApplyCharTypePadding extends Rule[LogicalPlan] {
  * rule right after the main resolution batch.
  */
 object RemoveTempResolvedColumn extends Rule[LogicalPlan] {
-  override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveExpressions {
-    case t: TempResolvedColumn => UnresolvedAttribute(t.nameParts)
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan.foreachUp {
+      // Having clause could be resolved as a Filter. When having func(column with wrong data type),
+      // the column could be wrapped by a TempResolvedColumn, e.g. mean(tempresolvedcolumn(t.c)).
+      // Because TempResolvedColumn can still preserve column data type, here is a chance to check
+      // if the data type matches with the required data type of the function. We can throw an error
+      // when data types mismatches.
+      case operator: Filter =>
+        operator.expressions.foreach(_.foreachUp {
+          case e: Expression if e.checkInputDataTypes().isFailure =>

Review comment:
       A straightforward fix is to change it to `case e: Expression if e.childrenResolved && e.checkInputDataTypes().isFailure`.
   
   A new idea is to not strip `TempResolvedColumn`, as `TempResolvedColumn` always means a failure. We can update `CheckAnalysis` to handle it, i.e. adding a new case after `case e: Expression if e.checkInputDataTypes().isFailure =>`
   ```
   case t: TempResolvedColumn =>
     val a = UnresolvedAttribute(t.nameParts)
     the same code that handles "case a: Attribute if !a.resolved"
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org