You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ge...@apache.org on 2022/03/30 09:21:14 UTC

[spark] branch branch-3.3 updated: [SPARK-38676][SQL] Provide SQL query context in runtime error message of Add/Subtract/Multiply

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

gengliang pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 7c523ea  [SPARK-38676][SQL] Provide SQL query context in runtime error message of Add/Subtract/Multiply
7c523ea is described below

commit 7c523eaff5c6c07f14832e7c2de80229656d6f9c
Author: Gengliang Wang <ge...@apache.org>
AuthorDate: Wed Mar 30 17:17:54 2022 +0800

    [SPARK-38676][SQL] Provide SQL query context in runtime error message of Add/Subtract/Multiply
    
    ### What changes were proposed in this pull request?
    
    Provide SQL query context in runtime error of Add/Subtract/Multiply if the data type is Int or Long. This is the first PR for improving the runtime error message. There are more to be done after we decide what the error message should be like.
    
    Before changes:
    ```
    > SELECT i.f1 - 100, i.f1 * smallint('2') AS x FROM INT4_TBL i
    java.lang.ArithmeticException
    integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
    ```
    After changes:
    ```
    > SELECT i.f1 - 100, i.f1 * smallint('2') AS x FROM INT4_TBL i
    java.lang.ArithmeticException
    integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
    
    == SQL(line 1, position 25) ==
    SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i
                             ^^^^^^^^^^^^^^^^^^^^
    ```
    
    ### Why are the changes needed?
    
    Provide SQL query context of runtime errors to users, so that they can understand it better.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, improve the runtime error message of Add/Subtract/Multiply
    
    ### How was this patch tested?
    
    UT
    
    Closes #35992 from gengliangwang/runtimeError.
    
    Authored-by: Gengliang Wang <ge...@apache.org>
    Signed-off-by: Gengliang Wang <ge...@apache.org>
    (cherry picked from commit 9f6aad407724997dc04a7689bb870d1d3ddd5526)
    Signed-off-by: Gengliang Wang <ge...@apache.org>
---
 .../sql/catalyst/expressions/arithmetic.scala      | 31 ++++++---
 .../apache/spark/sql/catalyst/trees/TreeNode.scala | 75 +++++++++++++++++++++-
 .../apache/spark/sql/catalyst/util/MathUtils.scala | 22 ++++++-
 .../spark/sql/errors/QueryExecutionErrors.scala    |  8 ++-
 .../expressions/ArithmeticExpressionSuite.scala    | 63 ++++++++++++++++++
 .../spark/sql/catalyst/trees/TreeNodeSuite.scala   | 32 +++++++++
 .../sql-tests/results/postgreSQL/int4.sql.out      | 18 ++++++
 .../sql-tests/results/postgreSQL/int8.sql.out      | 12 ++++
 8 files changed, 246 insertions(+), 15 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 88a3861..7251e47 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -268,19 +268,16 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant {
            |${ev.value} = (${CodeGenerator.javaType(dataType)})($tmpResult);
          """.stripMargin
       })
-    case IntegerType | LongType =>
+    case IntegerType | LongType if failOnError && exactMathMethod.isDefined =>
       nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
-        val operation = if (failOnError && exactMathMethod.isDefined) {
-          val mathUtils = MathUtils.getClass.getCanonicalName.stripSuffix("$")
-          s"$mathUtils.${exactMathMethod.get}($eval1, $eval2)"
-        } else {
-          s"$eval1 $symbol $eval2"
-        }
+        val errorContext = ctx.addReferenceObj("errCtx", origin.context)
+        val mathUtils = MathUtils.getClass.getCanonicalName.stripSuffix("$")
         s"""
-           |${ev.value} = $operation;
+           |${ev.value} = $mathUtils.${exactMathMethod.get}($eval1, $eval2, $errorContext);
          """.stripMargin
       })
-    case DoubleType | FloatType =>
+
+    case IntegerType | LongType | DoubleType | FloatType =>
       // When Double/Float overflows, there can be 2 cases:
       // - precision loss: according to SQL standard, the number is truncated;
       // - returns (+/-)Infinite: same behavior also other DBs have (e.g. Postgres)
@@ -333,6 +330,10 @@ case class Add(
       MathUtils.addExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long])
     case _: YearMonthIntervalType =>
       MathUtils.addExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int])
+    case _: IntegerType if failOnError =>
+      MathUtils.addExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int], origin.context)
+    case _: LongType if failOnError =>
+      MathUtils.addExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long], origin.context)
     case _ => numeric.plus(input1, input2)
   }
 
@@ -379,6 +380,10 @@ case class Subtract(
       MathUtils.subtractExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long])
     case _: YearMonthIntervalType =>
       MathUtils.subtractExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int])
+    case _: IntegerType if failOnError =>
+      MathUtils.subtractExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int], origin.context)
+    case _: LongType if failOnError =>
+      MathUtils.subtractExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long], origin.context)
     case _ => numeric.minus(input1, input2)
   }
 
@@ -411,7 +416,13 @@ case class Multiply(
 
   private lazy val numeric = TypeUtils.getNumeric(dataType, failOnError)
 
-  protected override def nullSafeEval(input1: Any, input2: Any): Any = numeric.times(input1, input2)
+  protected override def nullSafeEval(input1: Any, input2: Any): Any = dataType match {
+    case _: IntegerType if failOnError =>
+      MathUtils.multiplyExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int], origin.context)
+    case _: LongType if failOnError =>
+      MathUtils.multiplyExact(input1.asInstanceOf[Long], input2.asInstanceOf[Long], origin.context)
+    case _ => numeric.times(input1, input2)
+  }
 
   override def exactMathMethod: Option[String] = Some("multiplyExact")
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 84d92c1..00690ab 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -64,7 +64,80 @@ case class Origin(
   stopIndex: Option[Int] = None,
   sqlText: Option[String] = None,
   objectType: Option[String] = None,
-  objectName: Option[String] = None)
+  objectName: Option[String] = None) {
+
+  /**
+   * The SQL query context of current node. For example:
+   * == SQL of VIEW v1(line 1, position 25) ==
+   * SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i
+   *                          ^^^^^^^^^^^^^^^
+   */
+  lazy val context: String = sqlText.map { text =>
+    val positionContext = if (line.isDefined && startPosition.isDefined) {
+      s"(line ${line.get}, position ${startPosition.get})"
+    } else {
+      ""
+    }
+    val objectContext = if (objectType.isDefined && objectName.isDefined) {
+      s" of ${objectType.get} ${objectName.get}"
+    } else {
+      ""
+    }
+    val builder = new StringBuilder
+    builder ++= s"\n== SQL$objectContext$positionContext ==\n"
+
+    val start = startIndex.getOrElse(0)
+    val stop = stopIndex.getOrElse(sqlText.get.length - 1)
+    // Ideally we should show all the lines which contains the SQL text context of the current node:
+    // [additional text] [current tree node] [additional text]
+    // However, we need to truncate the additional text in case it is too long. The following
+    // variable is to define the max length of additional text.
+    val maxExtraContextLength = 32
+    val truncatedText = "..."
+    var lineStartIndex = start
+    // Collect the SQL text within the starting line of current Node.
+    // The text is truncated if it is too long.
+    while(lineStartIndex >= 0 &&
+      start - lineStartIndex <= maxExtraContextLength &&
+      text.charAt(lineStartIndex) != '\n') {
+      lineStartIndex -= 1
+    }
+    val startTruncated = start - lineStartIndex > maxExtraContextLength
+    var currentIndex = lineStartIndex
+    if (startTruncated) {
+      currentIndex -= truncatedText.length
+    }
+
+    var lineStopIndex = stop
+    // Collect the SQL text within the ending line of current Node.
+    // The text is truncated if it is too long.
+    while(lineStopIndex < text.length &&
+      lineStopIndex - stop <= maxExtraContextLength &&
+      text.charAt(lineStopIndex) != '\n') {
+      lineStopIndex += 1
+    }
+    val stopTruncated = lineStopIndex - stop > maxExtraContextLength
+
+    val subText = (if (startTruncated) truncatedText else "") +
+      text.substring(lineStartIndex + 1, lineStopIndex) +
+      (if (stopTruncated) truncatedText else "")
+    val lines = subText.split("\n")
+    lines.foreach { lineText =>
+      builder ++= lineText + "\n"
+      currentIndex += 1
+      (0 until lineText.length).foreach { _ =>
+        if (currentIndex < start) {
+          builder ++= " "
+        } else if (currentIndex >= start && currentIndex <= stop) {
+          builder ++= "^"
+        }
+        currentIndex += 1
+      }
+      builder ++= "\n"
+    }
+    builder.result()
+  }.getOrElse("")
+}
 
 /**
  * Provides a location for TreeNodes to ask about the context of their origin.  For example, which
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala
index 14c1ea1..f96c9fb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/MathUtils.scala
@@ -26,16 +26,34 @@ object MathUtils {
 
   def addExact(a: Int, b: Int): Int = withOverflow(Math.addExact(a, b))
 
+  def addExact(a: Int, b: Int, errorContext: String): Int =
+    withOverflow(Math.addExact(a, b), errorContext = errorContext)
+
   def addExact(a: Long, b: Long): Long = withOverflow(Math.addExact(a, b))
 
+  def addExact(a: Long, b: Long, errorContext: String): Long =
+    withOverflow(Math.addExact(a, b), errorContext = errorContext)
+
   def subtractExact(a: Int, b: Int): Int = withOverflow(Math.subtractExact(a, b))
 
+  def subtractExact(a: Int, b: Int, errorContext: String): Int =
+    withOverflow(Math.subtractExact(a, b), errorContext = errorContext)
+
   def subtractExact(a: Long, b: Long): Long = withOverflow(Math.subtractExact(a, b))
 
+  def subtractExact(a: Long, b: Long, errorContext: String): Long =
+    withOverflow(Math.subtractExact(a, b), errorContext = errorContext)
+
   def multiplyExact(a: Int, b: Int): Int = withOverflow(Math.multiplyExact(a, b))
 
+  def multiplyExact(a: Int, b: Int, errorContext: String): Int =
+    withOverflow(Math.multiplyExact(a, b), errorContext = errorContext)
+
   def multiplyExact(a: Long, b: Long): Long = withOverflow(Math.multiplyExact(a, b))
 
+  def multiplyExact(a: Long, b: Long, errorContext: String): Long =
+    withOverflow(Math.multiplyExact(a, b), errorContext = errorContext)
+
   def negateExact(a: Int): Int = withOverflow(Math.negateExact(a))
 
   def negateExact(a: Long): Long = withOverflow(Math.negateExact(a))
@@ -50,12 +68,12 @@ object MathUtils {
 
   def floorMod(a: Long, b: Long): Long = withOverflow(Math.floorMod(a, b))
 
-  private def withOverflow[A](f: => A, hint: String = ""): A = {
+  private def withOverflow[A](f: => A, hint: String = "", errorContext: String = ""): A = {
     try {
       f
     } catch {
       case e: ArithmeticException =>
-        throw QueryExecutionErrors.arithmeticOverflowError(e.getMessage, hint)
+        throw QueryExecutionErrors.arithmeticOverflowError(e.getMessage, hint, errorContext)
     }
   }
 }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index c6a69e4..8372150 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -441,10 +441,14 @@ object QueryExecutionErrors {
       s"to false to bypass this error.")
   }
 
-  def arithmeticOverflowError(message: String, hint: String = ""): ArithmeticException = {
+  def arithmeticOverflowError(
+      message: String,
+      hint: String = "",
+      errorContext: String = ""): ArithmeticException = {
     val alternative = if (hint.nonEmpty) s" To return NULL instead, use '$hint'." else ""
     new ArithmeticException(s"$message.$alternative If necessary set " +
-      s"${SQLConf.ANSI_ENABLED.key} to false (except for ANSI interval type) to bypass this error.")
+      s"${SQLConf.ANSI_ENABLED.key} to false (except for ANSI interval type) to bypass this " +
+      "error." + errorContext)
   }
 
   def unaryMinusCauseOverflowError(originValue: AnyVal): ArithmeticException = {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
index 522313f..c992b4d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
@@ -27,6 +27,8 @@ import org.apache.spark.sql.catalyst.analysis.DecimalPrecision
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
+import org.apache.spark.sql.catalyst.trees.Origin
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 
@@ -71,6 +73,34 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
     }
   }
 
+  private def getMaxValue(dt: DataType): Literal = dt match {
+    case IntegerType => Literal.create(Int.MaxValue, IntegerType)
+    case LongType => Literal.create(Long.MaxValue, LongType)
+    case _ => s"Fail to find the max value of $dt"
+  }
+
+  private def getMinValue(dt: DataType): Literal = dt match {
+    case IntegerType => Literal.create(Int.MinValue, IntegerType)
+    case LongType => Literal.create(Long.MinValue, LongType)
+    case _ => s"Fail to find the min value of $dt"
+  }
+
+  test("Add: Overflow exception should contain SQL text context") {
+    Seq(IntegerType, LongType).foreach { dt =>
+      val maxValue = getMaxValue(dt)
+      val query = s"${maxValue.sql} + ${maxValue.sql}"
+      val o = Origin(
+        line = Some(1),
+        startPosition = Some(7),
+        startIndex = Some(7),
+        sqlText = Some(s"select $query"))
+      withOrigin(o) {
+        val expr = Add(maxValue, maxValue, failOnError = true)
+        checkExceptionInExpression[ArithmeticException](expr, EmptyRow, query)
+      }
+    }
+  }
+
   test("- (UnaryMinus)") {
     testNumericDataTypes { convert =>
       val input = Literal(convert(1))
@@ -138,6 +168,23 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
     }
   }
 
+  test("Minus: Overflow exception should contain SQL text context") {
+    Seq(IntegerType, LongType).foreach { dt =>
+      val minValue = getMinValue(dt)
+      val maxValue = getMaxValue(dt)
+      val query = s"${minValue.sql} - ${maxValue.sql}"
+      val o = Origin(
+        line = Some(1),
+        startPosition = Some(7),
+        startIndex = Some(7),
+        sqlText = Some(s"select $query"))
+      withOrigin(o) {
+        val expr = Subtract(minValue, maxValue, failOnError = true)
+        checkExceptionInExpression[ArithmeticException](expr, EmptyRow, query)
+      }
+    }
+  }
+
   test("* (Multiply)") {
     testNumericDataTypes { convert =>
       val left = Literal(convert(1))
@@ -160,6 +207,22 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
     }
   }
 
+  test("Multiply: Overflow exception should contain SQL text context") {
+    Seq(IntegerType, LongType).foreach { dt =>
+      val maxValue = getMaxValue(dt)
+      val query = s"${maxValue.sql} * ${maxValue.sql}"
+      val o = Origin(
+        line = Some(1),
+        startPosition = Some(7),
+        startIndex = Some(7),
+        sqlText = Some(s"select $query"))
+      withOrigin(o) {
+        val expr = Multiply(maxValue, maxValue, failOnError = true)
+        checkExceptionInExpression[ArithmeticException](expr, EmptyRow, query)
+      }
+    }
+  }
+
   private def testDecimalAndDoubleType(testFunc: (Int => Any) => Unit): Unit = {
     testFunc(_.toDouble)
     testFunc(Decimal(_))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
index b6087c5..4bdb3f7 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
@@ -856,4 +856,36 @@ class TreeNodeSuite extends SparkFunSuite with SQLHelper {
     val node = Node(Set("second", "first"), Seq(Set(3, 1), Set(2, 1)))
     assert(node.argString(10) == "{first, second}, [{1, 3}, {1, 2}]")
   }
+
+  test("SPARK-38676: truncate before/after sql text if too long") {
+    val text =
+      """
+        |
+        |SELECT
+        |1234567890 + 1234567890 + 1234567890, cast('a'
+        |as /* comment */
+        |int), 1234567890 + 1234567890 + 1234567890
+        |as foo
+        |""".stripMargin
+    val origin = Origin(
+      line = Some(3),
+      startPosition = Some(38),
+      startIndex = Some(47),
+      stopIndex = Some(77),
+      sqlText = Some(text),
+      objectType = Some("VIEW"),
+      objectName = Some("some_view"))
+    val expected =
+      """
+        |== SQL of VIEW some_view(line 3, position 38) ==
+        |...7890 + 1234567890 + 1234567890, cast('a'
+        |                                   ^^^^^^^^
+        |as /* comment */
+        |^^^^^^^^^^^^^^^^
+        |int), 1234567890 + 1234567890 + 12345...
+        |^^^^^
+        |""".stripMargin
+
+    assert(origin.context == expected)
+  }
 }
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out
index d5637d9..993fe44 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int4.sql.out
@@ -201,6 +201,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -221,6 +224,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -241,6 +247,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -262,6 +271,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -283,6 +295,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -304,6 +319,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 integer overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 25) ==
+SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i
+                         ^^^^^^^^^^^^^^^
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
index 427e89a..af30653c 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/int8.sql.out
@@ -393,6 +393,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 long overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 28) ==
+SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL
+                            ^^^^^^^
 
 
 -- !query
@@ -743,6 +746,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 long overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 7) ==
+SELECT bigint((-9223372036854775808)) * bigint((-1))
+       ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -768,6 +774,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 long overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 7) ==
+SELECT bigint((-9223372036854775808)) * int((-1))
+       ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query
@@ -793,6 +802,9 @@ struct<>
 -- !query output
 java.lang.ArithmeticException
 long overflow. If necessary set spark.sql.ansi.enabled to false (except for ANSI interval type) to bypass this error.
+== SQL(line 1, position 7) ==
+SELECT bigint((-9223372036854775808)) * smallint((-1))
+       ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
 
 
 -- !query

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