This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 08a4ade8ba8 [SPARK-39106][SQL] Correct conditional expression constant 
folding
08a4ade8ba8 is described below

commit 08a4ade8ba881589da0741b3ffacd3304dc1e9b5
Author: ulysses-you <ulyssesyo...@gmail.com>
AuthorDate: Tue May 10 20:19:36 2022 +0800

    [SPARK-39106][SQL] Correct conditional expression constant folding
    
    ### What changes were proposed in this pull request?
    
    - add try-catch when we fold children inside `ConditionalExpression` if 
it's not foldable
    - mark `CaseWhen` and `If` as foldable if it's children are foldable
    
    ### Why are the changes needed?
    
    For a conditional expression, we should add a try-catch to partially fold 
the constant inside it's children because some bracnhes may no be evaluated at 
runtime. For example if c1 or c2 is not null, the last branch should be never 
hit:
    ```sql
    SELECT COALESCE(c1, c2, 1/0);
    ```
    Besides, for CaseWhen and If, we should mark it as foldable if it's 
children are foldable. It is safe since the both non-codegen and codegen code 
path have already respected the evaluation order.
    
    ### Does this PR introduce _any_ user-facing change?
    
    yes, bug fix
    
    ### How was this patch tested?
    
    add more test in sql file
    
    Closes #36468 from ulysses-you/SPARK-39106.
    
    Authored-by: ulysses-you <ulyssesyo...@gmail.com>
    Signed-off-by: Wenchen Fan <wenc...@databricks.com>
---
 .../sql/catalyst/expressions/Expression.scala      |   2 +
 .../expressions/conditionalExpressions.scala       |   1 +
 .../sql/catalyst/expressions/nullExpressions.scala |   3 -
 .../spark/sql/catalyst/optimizer/expressions.scala |  56 +++++++---
 .../catalyst/optimizer/ConstantFoldingSuite.scala  |  38 +++++++
 .../inputs/ansi/conditional-functions.sql          |  19 +++-
 .../resources/sql-tests/inputs/postgreSQL/case.sql |   1 -
 .../sql-tests/inputs/udf/postgreSQL/udf-case.sql   |   6 +-
 .../results/ansi/conditional-functions.sql.out     | 115 ++++++++++++++++++++-
 .../sql-tests/results/postgreSQL/case.sql.out      |  27 ++---
 .../results/udf/postgreSQL/udf-case.sql.out        |  30 +-----
 11 files changed, 224 insertions(+), 74 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 30b6773ce1c..e1d8c2e43e2 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -459,6 +459,8 @@ trait Nondeterministic extends Expression {
  * All optimization should be careful with the evaluation order.
  */
 trait ConditionalExpression extends Expression {
+  final override def foldable: Boolean = children.forall(_.foldable)
+
   /**
    * Return the children expressions which can always be hit at runtime.
    */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
index 5dacabd646d..7213440bebe 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala
@@ -48,6 +48,7 @@ case class If(predicate: Expression, trueValue: Expression, 
falseValue: Expressi
   override def second: Expression = trueValue
   override def third: Expression = falseValue
   override def nullable: Boolean = trueValue.nullable || falseValue.nullable
+
   /**
    * Only the condition expression will always be evaluated.
    */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
index 8f59ab5b249..8d171c2c663 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
@@ -53,9 +53,6 @@ case class Coalesce(children: Seq[Expression])
   /** Coalesce is nullable if all of its children are nullable, or if it has 
no children. */
   override def nullable: Boolean = children.forall(_.nullable)
 
-  // Coalesce is foldable if all children are foldable.
-  override def foldable: Boolean = children.forall(_.foldable)
-
   final override val nodePatterns: Seq[TreePattern] = Seq(COALESCE)
 
   override def checkInputDataTypes(): TypeCheckResult = {
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 7060a1ff1d0..8971f0c70af 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer
 
 import scala.collection.immutable.HashSet
 import scala.collection.mutable.{ArrayBuffer, Stack}
+import scala.util.control.NonFatal
 
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.expressions.{MultiLikeBase, _}
@@ -28,7 +29,7 @@ import 
org.apache.spark.sql.catalyst.expressions.objects.AssertNotNull
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.trees.AlwaysProcess
+import org.apache.spark.sql.catalyst.trees.{AlwaysProcess, TreeNodeTag}
 import org.apache.spark.sql.catalyst.trees.TreePattern._
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.UTF8String
@@ -43,6 +44,9 @@ import org.apache.spark.unsafe.types.UTF8String
  * equivalent [[Literal]] values.
  */
 object ConstantFolding extends Rule[LogicalPlan] {
+  // This tag is for avoid repeatedly evaluating expression inside conditional 
expression
+  // which has already failed to evaluate before.
+  private[sql] val FAILED_TO_EVALUATE = TreeNodeTag[Unit]("FAILED_TO_EVALUATE")
 
   private def hasNoSideEffect(e: Expression): Boolean = e match {
     case _: Attribute => true
@@ -52,22 +56,42 @@ object ConstantFolding extends Rule[LogicalPlan] {
     case _ => false
   }
 
+  private def constantFolding(
+      e: Expression,
+      isConditionalBranch: Boolean = false): Expression = e match {
+    case c: ConditionalExpression if !c.foldable =>
+      c.mapChildren(constantFolding(_, isConditionalBranch = true))
+
+    // Skip redundant folding of literals. This rule is technically not 
necessary. Placing this
+    // here avoids running the next rule for Literal values, which would 
create a new Literal
+    // object and running eval unnecessarily.
+    case l: Literal => l
+
+    case Size(c: CreateArray, _) if c.children.forall(hasNoSideEffect) =>
+      Literal(c.children.length)
+    case Size(c: CreateMap, _) if c.children.forall(hasNoSideEffect) =>
+      Literal(c.children.length / 2)
+
+    case e if e.getTagValue(FAILED_TO_EVALUATE).isDefined => e
+
+    // Fold expressions that are foldable.
+    case e if e.foldable =>
+      try {
+        Literal.create(e.eval(EmptyRow), e.dataType)
+      } catch {
+        case NonFatal(_) if isConditionalBranch =>
+          // When doing constant folding inside conditional expressions, we 
should not fail
+          // during expression evaluation, as the branch we are evaluating may 
not be reached at
+          // runtime, and we shouldn't fail the query, to match the original 
behavior.
+          e.setTagValue(FAILED_TO_EVALUATE, ())
+          e
+      }
+
+    case other => other.mapChildren(constantFolding(_, isConditionalBranch))
+  }
+
   def apply(plan: LogicalPlan): LogicalPlan = 
plan.transformWithPruning(AlwaysProcess.fn, ruleId) {
-    case q: LogicalPlan => q.transformExpressionsDownWithPruning(
-      AlwaysProcess.fn, ruleId) {
-      // Skip redundant folding of literals. This rule is technically not 
necessary. Placing this
-      // here avoids running the next rule for Literal values, which would 
create a new Literal
-      // object and running eval unnecessarily.
-      case l: Literal => l
-
-      case Size(c: CreateArray, _) if c.children.forall(hasNoSideEffect) =>
-        Literal(c.children.length)
-      case Size(c: CreateMap, _) if c.children.forall(hasNoSideEffect) =>
-        Literal(c.children.length / 2)
-
-      // Fold expressions that are foldable.
-      case e if e.foldable => Literal.create(e.eval(EmptyRow), e.dataType)
-    }
+    case q: LogicalPlan => q.mapExpressions(constantFolding(_))
   }
 }
 
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
index bce138beb53..39150648d29 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import org.apache.spark.SparkArithmeticException
+import org.apache.spark.sql.Row
 import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, 
UnresolvedExtractValue}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.dsl.plans._
@@ -25,6 +27,7 @@ import 
org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke}
 import org.apache.spark.sql.catalyst.plans.PlanTest
 import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
 import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.ByteArray
 
@@ -331,4 +334,39 @@ class ConstantFoldingSuite extends PlanTest {
 
     comparePlans(optimized, correctAnswer)
   }
+
+  test("SPARK-39106: Correct conditional expression constant folding") {
+    val t = LocalRelation.fromExternalRows(
+      $"c".double :: Nil,
+      Row(1d) :: Row(null) :: Row(Double.NaN) :: Nil)
+
+    withSQLConf(SQLConf.ANSI_ENABLED.key -> "true") {
+      // conditional expression is foldable, throw exception during query 
compilation
+      Seq(
+        t.select(CaseWhen((Divide(1, 0) === 1, Add(1, 0)) :: Nil, Subtract(1, 
0))),
+        t.select(If(Divide(1, 0) === 1, Add(1, 0), Add(1, 0))),
+        t.select(Coalesce(Divide(1, 0) :: Add(1, 0) :: Nil)),
+        t.select(NaNvl(Divide(1, 0), Add(1, 0)))
+      ).foreach { query =>
+        intercept[SparkArithmeticException] {
+          Optimize.execute(query.analyze)
+        }
+      }
+
+      // conditional expression is not foldable, suppress the exception during 
query compilation
+      Seq(
+        t.select(CaseWhen(($"c" === 1d, Divide(1, 0)) :: Nil, 1d)),
+        t.select(If($"c" === 1d, Divide(1, 0), 1d)),
+        t.select(Coalesce($"c" :: Divide(1, 0) :: Nil)),
+        t.select(NaNvl($"c", Divide(1, 0)))
+      ).foreach { query =>
+        val optimized = Optimize.execute(query.analyze)
+        val failedToEvaluated = optimized.expressions.flatMap(_.collect {
+          case e: Expression if 
e.getTagValue(ConstantFolding.FAILED_TO_EVALUATE).isDefined => e
+        })
+        assert(failedToEvaluated.size == 1)
+        comparePlans(query.analyze, optimized)
+      }
+    }
+  }
 }
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql 
b/sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql
index 5c548b1e9c4..ba8f0ffe7f1 100644
--- 
a/sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql
+++ 
b/sql/core/src/test/resources/sql-tests/inputs/ansi/conditional-functions.sql
@@ -1,6 +1,21 @@
 -- Tests for conditional functions
-CREATE TABLE t USING PARQUET AS SELECT c1, c2 FROM VALUES(1, 0),(2, 1) AS 
t(c1, c2);
 
-SELECT nanvl(c1, c1/c2 + c1/c2) FROM t;
+CREATE TABLE t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 
1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2);
+
+SELECT nanvl(c2, c1/c2 + c1/c2) FROM t;
+SELECT nanvl(c2, 1/0) FROM t;
+SELECT nanvl(1-0, 1/0) FROM t;
+
+SELECT if(c2 >= 0, 1-0, 1/0) from t;
+SELECT if(1 == 1, 1, 1/0);
+SELECT if(1 != 1, 1/0, 1);
+
+SELECT coalesce(c2, 1/0) from t;
+SELECT coalesce(1, 1/0);
+SELECT coalesce(null, 1, 1/0);
+
+SELECT case when c2 >= 0 then 1 else 1/0 end from t;
+SELECT case when 1 < 2 then 1 else 1/0 end;
+SELECT case when 1 > 2 then 1/0 else 1 end;
 
 DROP TABLE IF EXISTS t;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql 
b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql
index b39ccb85fb3..e8129b8cfaf 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/postgreSQL/case.sql
@@ -65,7 +65,6 @@ SELECT '7' AS `None`,
   CASE WHEN rand() < 0 THEN 1
   END AS `NULL on no matches`;
 
--- [SPARK-33008] Spark SQL throws an exception
 -- Constant-expression folding shouldn't evaluate unreachable subexpressions
 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END;
 SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END;
diff --git 
a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql 
b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql
index 5322c1b5024..4ee0dba8fa9 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/udf/postgreSQL/udf-case.sql
@@ -67,14 +67,12 @@ SELECT '7' AS `None`,
   CASE WHEN rand() < udf(0) THEN 1
   END AS `NULL on no matches`;
 
--- [SPARK-33008] Spark SQL throws an exception
 -- Constant-expression folding shouldn't evaluate unreachable subexpressions
 SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END;
 SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END;
 
--- However we do not currently suppress folding of potentially
--- reachable subexpressions
-SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl;
+-- SPARK-39122: Python UDF does not follow the conditional expression 
evaluation order
+-- SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl;
 
 -- Test for cases involving untyped literals in test expression
 SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END;
diff --git 
a/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out
index d3af659fc48..6a4f694f4d7 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/ansi/conditional-functions.sql.out
@@ -1,9 +1,9 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 3
+-- Number of queries: 14
 
 
 -- !query
-CREATE TABLE t USING PARQUET AS SELECT c1, c2 FROM VALUES(1, 0),(2, 1) AS 
t(c1, c2)
+CREATE TABLE t USING PARQUET AS SELECT c1, c2 FROM VALUES(1d, 0),(2d, 
1),(null, 1),(CAST('NaN' AS DOUBLE), 0) AS t(c1, c2)
 -- !query schema
 struct<>
 -- !query output
@@ -11,12 +11,117 @@ struct<>
 
 
 -- !query
-SELECT nanvl(c1, c1/c2 + c1/c2) FROM t
+SELECT nanvl(c2, c1/c2 + c1/c2) FROM t
 -- !query schema
-struct<nanvl(c1, ((c1 / c2) + (c1 / c2))):double>
+struct<nanvl(c2, ((c1 / c2) + (c1 / c2))):double>
+-- !query output
+0.0
+0.0
+1.0
+1.0
+
+
+-- !query
+SELECT nanvl(c2, 1/0) FROM t
+-- !query schema
+struct<nanvl(c2, (1 / 0)):double>
+-- !query output
+0.0
+0.0
+1.0
+1.0
+
+
+-- !query
+SELECT nanvl(1-0, 1/0) FROM t
+-- !query schema
+struct<nanvl((1 - 0), (1 / 0)):double>
+-- !query output
+1.0
+1.0
+1.0
+1.0
+
+
+-- !query
+SELECT if(c2 >= 0, 1-0, 1/0) from t
+-- !query schema
+struct<(IF((c2 >= 0), (1 - 0), (1 / 0))):double>
+-- !query output
+1.0
+1.0
+1.0
+1.0
+
+
+-- !query
+SELECT if(1 == 1, 1, 1/0)
+-- !query schema
+struct<(IF((1 = 1), 1, (1 / 0))):double>
+-- !query output
+1.0
+
+
+-- !query
+SELECT if(1 != 1, 1/0, 1)
+-- !query schema
+struct<(IF((NOT (1 = 1)), (1 / 0), 1)):double>
+-- !query output
+1.0
+
+
+-- !query
+SELECT coalesce(c2, 1/0) from t
+-- !query schema
+struct<coalesce(c2, (1 / 0)):double>
+-- !query output
+0.0
+0.0
+1.0
+1.0
+
+
+-- !query
+SELECT coalesce(1, 1/0)
+-- !query schema
+struct<coalesce(1, (1 / 0)):double>
+-- !query output
+1.0
+
+
+-- !query
+SELECT coalesce(null, 1, 1/0)
+-- !query schema
+struct<coalesce(NULL, 1, (1 / 0)):double>
+-- !query output
+1.0
+
+
+-- !query
+SELECT case when c2 >= 0 then 1 else 1/0 end from t
+-- !query schema
+struct<CASE WHEN (c2 >= 0) THEN 1 ELSE (1 / 0) END:double>
+-- !query output
+1.0
+1.0
+1.0
+1.0
+
+
+-- !query
+SELECT case when 1 < 2 then 1 else 1/0 end
+-- !query schema
+struct<CASE WHEN (1 < 2) THEN 1 ELSE (1 / 0) END:double>
+-- !query output
+1.0
+
+
+-- !query
+SELECT case when 1 > 2 then 1/0 else 1 end
+-- !query schema
+struct<CASE WHEN (1 > 2) THEN (1 / 0) ELSE 1 END:double>
 -- !query output
 1.0
-2.0
 
 
 -- !query
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out
index e0e59cb28bd..603df103846 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/case.sql.out
@@ -176,37 +176,28 @@ struct<None:string,NULL on no matches:int>
 -- !query
 SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
 -- !query schema
-struct<>
+struct<CASE WHEN (1 = 0) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) 
END:double>
 -- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 26) ==
-SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
-                          ^^^
+1.0
 
 
 -- !query
 SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END
 -- !query schema
-struct<>
+struct<CASE WHEN (1 = 0) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) 
END:double>
 -- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 26) ==
-SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END
-                          ^^^
+1.0
 
 
 -- !query
 SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl
 -- !query schema
-struct<>
+struct<CASE WHEN (i > 100) THEN (1 / 0) ELSE 0 END:double>
 -- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 30) ==
-SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl
-                              ^^^
+0.0
+0.0
+0.0
+0.0
 
 
 -- !query
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out
index 9f325325906..7c0a37ee667 100755
--- 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-case.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 35
+-- Number of queries: 34
 
 
 -- !query
@@ -176,37 +176,17 @@ struct<None:string,NULL on no matches:int>
 -- !query
 SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
 -- !query schema
-struct<>
+struct<CASE WHEN udf((1 = 0)) THEN (1 / 0) WHEN (1 = 1) THEN 1 ELSE (2 / 0) 
END:double>
 -- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 31) ==
-SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END
-                               ^^^
+1.0
 
 
 -- !query
 SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END
 -- !query schema
-struct<>
+struct<CASE WHEN (1 = 0) THEN (1 / udf(0)) WHEN (1 = 1) THEN 1 ELSE (2 / 0) 
END:double>
 -- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 54) ==
-...HEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END
-                                   ^^^
-
-
--- !query
-SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl
--- !query schema
-struct<>
--- !query output
-org.apache.spark.SparkArithmeticException
-[DIVIDE_BY_ZERO] Division by zero. To return NULL instead, use `try_divide`. 
If necessary set "spark.sql.ansi.enabled" to false (except for ANSI interval 
type) to bypass this error.
-== SQL(line 1, position 34) ==
-...LECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl
-                                   ^^^
+1.0
 
 
 -- !query


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to