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

maxgekk 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 d503c47c9e3 [SPARK-41019][SQL] Provide a query context to 
`failAnalysis()`
d503c47c9e3 is described below

commit d503c47c9e3a4f1e815bae3b57feadd5568ca21a
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Mon Nov 7 13:03:17 2022 +0300

    [SPARK-41019][SQL] Provide a query context to `failAnalysis()`
    
    ### What changes were proposed in this pull request?
    In the PR, I propose to invoke `AnalysisErrorAt.failAnalysis()` instead of 
`CheckAnalysis.failAnalysis()` because the first one captures the query context 
and passes it to `AnalysisException`.
    
    ### Why are the changes needed?
    To provide additional info as a query context to users. This should improve 
user experience with Spark SQL.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    By running the modified test suites:
    ```
    $ PYSPARK_PYTHON=python3 build/sbt "sql/testOnly 
org.apache.spark.sql.SQLQueryTestSuite"
    ```
    
    Closes #38514 from MaxGekk/provide-context-failAnalysis.
    
    Authored-by: Max Gekk <max.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 .../sql/catalyst/analysis/CheckAnalysis.scala      | 106 ++--
 .../resources/sql-tests/results/except-all.sql.out |  18 +-
 .../sql-tests/results/group-analytics.sql.out      |  18 +-
 .../sql-tests/results/group-by-filter.sql.out      |   9 +-
 .../resources/sql-tests/results/group-by.sql.out   |  63 ++-
 .../sql-tests/results/intersect-all.sql.out        |  18 +-
 .../test/resources/sql-tests/results/limit.sql.out |  45 +-
 .../sql-tests/results/percentiles.sql.out          | 108 +++-
 .../test/resources/sql-tests/results/pivot.sql.out |   9 +-
 .../results/postgreSQL/aggregates_part3.sql.out    |   9 +-
 .../sql-tests/results/postgreSQL/limit.sql.out     |  18 +-
 .../results/postgreSQL/select_having.sql.out       |   9 +-
 .../results/postgreSQL/window_part3.sql.out        |  18 +-
 .../negative-cases/invalid-correlation.sql.out     |  18 +-
 .../native/widenSetOperationTypes.sql.out          | 630 ++++++++++++++++++---
 .../results/udaf/udaf-group-by-ordinal.sql.out     |   9 +-
 .../sql-tests/results/udaf/udaf-group-by.sql.out   |   9 +-
 .../udf/postgreSQL/udf-aggregates_part3.sql.out    |   9 +-
 .../udf/postgreSQL/udf-select_having.sql.out       |   9 +-
 .../sql-tests/results/udf/udf-except-all.sql.out   |  18 +-
 .../results/udf/udf-group-analytics.sql.out        |  18 +-
 .../sql-tests/results/udf/udf-group-by.sql.out     |  54 +-
 .../results/udf/udf-intersect-all.sql.out          |  18 +-
 .../sql-tests/results/udf/udf-pivot.sql.out        |   9 +-
 24 files changed, 1064 insertions(+), 185 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index 0b688dc5f7c..544bb3cc301 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -82,24 +82,24 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
   private def checkLimitLikeClause(name: String, limitExpr: Expression): Unit 
= {
     limitExpr match {
-      case e if !e.foldable => failAnalysis(
+      case e if !e.foldable => limitExpr.failAnalysis(
         errorClass = "_LEGACY_ERROR_TEMP_2400",
         messageParameters = Map(
           "name" -> name,
           "limitExpr" -> limitExpr.sql))
-      case e if e.dataType != IntegerType => failAnalysis(
+      case e if e.dataType != IntegerType => limitExpr.failAnalysis(
         errorClass = "_LEGACY_ERROR_TEMP_2401",
         messageParameters = Map(
           "name" -> name,
           "dataType" -> e.dataType.catalogString))
       case e =>
         e.eval() match {
-          case null => failAnalysis(
+          case null => limitExpr.failAnalysis(
             errorClass = "_LEGACY_ERROR_TEMP_2402",
             messageParameters = Map(
               "name" -> name,
               "limitExpr" -> limitExpr.sql))
-          case v: Int if v < 0 => failAnalysis(
+          case v: Int if v < 0 => limitExpr.failAnalysis(
             errorClass = "_LEGACY_ERROR_TEMP_2403",
             messageParameters = Map(
               "name" -> name,
@@ -189,12 +189,12 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
           case r @ ResolvedTable(_, _, table, _) => table match {
             case t: SupportsPartitionManagement =>
               if (t.partitionSchema.isEmpty) {
-                failAnalysis(
+                r.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2404",
                   messageParameters = Map("name" -> r.name))
               }
             case _ =>
-              failAnalysis(
+              r.failAnalysis(
                 errorClass = "_LEGACY_ERROR_TEMP_2405",
                 messageParameters = Map("name" -> r.name))
           }
@@ -254,7 +254,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
             }
 
           case c: Cast if !c.resolved =>
-            failAnalysis(
+            c.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2406",
               messageParameters = Map(
                 "srcType" -> c.child.dataType.catalogString,
@@ -264,26 +264,26 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
               "\nReplacement is unresolved: " + e.replacement)
 
           case g: Grouping =>
-            failAnalysis(errorClass = "_LEGACY_ERROR_TEMP_2445", 
messageParameters = Map.empty)
+            g.failAnalysis(errorClass = "_LEGACY_ERROR_TEMP_2445", 
messageParameters = Map.empty)
           case g: GroupingID =>
-            failAnalysis(errorClass = "_LEGACY_ERROR_TEMP_2407", 
messageParameters = Map.empty)
+            g.failAnalysis(errorClass = "_LEGACY_ERROR_TEMP_2407", 
messageParameters = Map.empty)
 
           case e: Expression if 
e.children.exists(_.isInstanceOf[WindowFunction]) &&
               !e.isInstanceOf[WindowExpression] && e.resolved =>
             val w = e.children.find(_.isInstanceOf[WindowFunction]).get
-            failAnalysis(
+            e.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2408",
               messageParameters = Map("w" -> w.toString))
 
           case w @ WindowExpression(AggregateExpression(_, _, true, _, _), _) 
=>
-            failAnalysis(
+            w.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2409",
               messageParameters = Map("w" -> w.toString))
 
           case w @ WindowExpression(wf: FrameLessOffsetWindowFunction,
             WindowSpecDefinition(_, order, frame: SpecifiedWindowFrame))
              if order.isEmpty || !frame.isOffset =>
-            failAnalysis(
+            w.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2410",
               messageParameters = Map(
                 "wf" -> wf.prettyName,
@@ -297,15 +297,14 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 _: PercentileCont | _: PercentileDisc | _: Median, _, _, _, _)
                 if w.windowSpec.orderSpec.nonEmpty || 
w.windowSpec.frameSpecification !=
                     SpecifiedWindowFrame(RowFrame, UnboundedPreceding, 
UnboundedFollowing) =>
-                failAnalysis(
+                agg.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2411",
-                  messageParameters = Map(
-                    "aggFunc" -> agg.aggregateFunction.prettyName))
+                  messageParameters = Map("aggFunc" -> 
agg.aggregateFunction.prettyName))
               case _: AggregateExpression | _: FrameLessOffsetWindowFunction |
                   _: AggregateWindowFunction => // OK
               case f: PythonUDF if PythonUDF.isWindowPandasUDF(f) => // OK
               case other =>
-                failAnalysis(
+                other.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2412",
                   messageParameters = Map("sqlExpr" -> other.toString))
             }
@@ -314,7 +313,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
             checkSubqueryExpression(operator, s)
 
           case e: ExpressionWithRandomSeed if !e.seedExpression.foldable =>
-            failAnalysis(
+            e.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2413",
               messageParameters = Map("argName" -> e.prettyName))
 
@@ -328,21 +327,21 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 if s.find(_.name == "end").map(_.dataType) == 
Some(TimestampType) =>
               case _: TimestampType =>
               case _ =>
-                failAnalysis(
+                etw.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2414",
                   messageParameters = Map(
                     "evName" -> etw.eventTime.name,
                     "evType" -> etw.eventTime.dataType.catalogString))
             }
           case f: Filter if f.condition.dataType != BooleanType =>
-            failAnalysis(
+            f.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2415",
               messageParameters = Map(
                 "filter" -> f.condition.sql,
                 "type" -> f.condition.dataType.catalogString))
 
           case j @ Join(_, _, _, Some(condition), _) if condition.dataType != 
BooleanType =>
-            failAnalysis(
+            j.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2416",
               messageParameters = Map(
                 "join" -> condition.sql,
@@ -350,7 +349,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case j @ AsOfJoin(_, _, _, Some(condition), _, _, _)
               if condition.dataType != BooleanType =>
-            failAnalysis(
+            j.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2417",
               messageParameters = Map(
                 "condition" -> condition.sql,
@@ -358,12 +357,12 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case j @ AsOfJoin(_, _, _, _, _, _, Some(toleranceAssertion)) =>
             if (!toleranceAssertion.foldable) {
-              failAnalysis(
+              j.failAnalysis(
                 errorClass = "_LEGACY_ERROR_TEMP_2418",
                 messageParameters = Map.empty)
             }
             if (!toleranceAssertion.eval().asInstanceOf[Boolean]) {
-              failAnalysis(
+              j.failAnalysis(
                 errorClass = "_LEGACY_ERROR_TEMP_2419",
                 messageParameters = Map.empty)
             }
@@ -378,14 +377,14 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 aggFunction.children.foreach { child =>
                   child.foreach {
                     case expr: Expression if 
AggregateExpression.isAggregate(expr) =>
-                      failAnalysis(
+                      expr.failAnalysis(
                         errorClass = "_LEGACY_ERROR_TEMP_2420",
                         messageParameters = Map.empty)
                     case other => // OK
                   }
 
                   if (!child.deterministic) {
-                    failAnalysis(
+                    child.failAnalysis(
                       errorClass = "_LEGACY_ERROR_TEMP_2421",
                       messageParameters = Map("sqlExpr" -> expr.sql))
                   }
@@ -395,7 +394,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 val aggExprs = aggregateExprs.filter(_.collect {
                   case a: AggregateExpression => a
                 }.nonEmpty)
-                failAnalysis(
+                e.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2422",
                   messageParameters = Map(
                     "sqlExpr" -> e.sql,
@@ -404,7 +403,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 throw QueryCompilationErrors.columnNotInGroupByClauseError(e)
               case s: ScalarSubquery
                   if s.children.nonEmpty && 
!groupingExprs.exists(_.semanticEquals(s)) =>
-                failAnalysis(
+                s.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2423",
                   messageParameters = Map("sqlExpr" -> s.sql))
               case e if groupingExprs.exists(_.semanticEquals(e)) => // OK
@@ -413,14 +412,14 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
             def checkValidGroupingExprs(expr: Expression): Unit = {
               if (expr.exists(_.isInstanceOf[AggregateExpression])) {
-                failAnalysis(
+                expr.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2424",
                   messageParameters = Map("sqlExpr" -> expr.sql))
               }
 
               // Check if the data type of expr is orderable.
               if (!RowOrdering.isOrderable(expr.dataType)) {
-                failAnalysis(
+                expr.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2425",
                   messageParameters = Map(
                     "sqlExpr" -> expr.sql,
@@ -431,7 +430,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 // This is just a sanity check, our analysis rule 
PullOutNondeterministic should
                 // already pull out those nondeterministic expressions and 
evaluate them in
                 // a Project node.
-                failAnalysis(
+                expr.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2426",
                   messageParameters = Map("sqlExpr" -> expr.sql))
               }
@@ -501,7 +500,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
           case Sort(orders, _, _) =>
             orders.foreach { order =>
               if (!RowOrdering.isOrderable(order.dataType)) {
-                failAnalysis(
+                order.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2427",
                   messageParameters = Map("type" -> 
order.dataType.catalogString))
               }
@@ -516,7 +515,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                 val limit = limitExpr.eval().asInstanceOf[Int]
                 val offset = offsetExpr.eval().asInstanceOf[Int]
                 if (Int.MaxValue - limit < offset) {
-                  failAnalysis(
+                  child.failAnalysis(
                     errorClass = "_LEGACY_ERROR_TEMP_2428",
                     messageParameters = Map(
                       "limit" -> limit.toString,
@@ -529,7 +528,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case Tail(limitExpr, _) => checkLimitLikeClause("tail", limitExpr)
 
-          case _: Union | _: SetOperation if operator.children.length > 1 =>
+          case e @ (_: Union | _: SetOperation) if operator.children.length > 
1 =>
             def dataTypes(plan: LogicalPlan): Seq[DataType] = 
plan.output.map(_.dataType)
             def ordinalNumber(i: Int): String = i match {
               case 0 => "first"
@@ -541,7 +540,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
             operator.children.tail.zipWithIndex.foreach { case (child, ti) =>
               // Check the number of columns
               if (child.output.length != ref.length) {
-                failAnalysis(
+                e.failAnalysis(
                   errorClass = "_LEGACY_ERROR_TEMP_2429",
                   messageParameters = Map(
                     "operator" -> operator.nodeName,
@@ -555,14 +554,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
               dataTypes(child).zip(ref).zipWithIndex.foreach { case ((dt1, 
dt2), ci) =>
                 // SPARK-18058: we shall not care about the nullability of 
columns
                 if (!dataTypesAreCompatibleFn(dt1, dt2)) {
-                  val errorMessage =
-                    s"""
-                       |${operator.nodeName} can only be performed on tables 
with compatible
-                       |column types. The ${ordinalNumber(ci)} column of the
-                       |${ordinalNumber(ti + 1)} table is ${dt1.catalogString} 
type which is not
-                       |compatible with ${dt2.catalogString} at the same 
column of the first table
-                    """.stripMargin.replace("\n", " ").trim()
-                  failAnalysis(
+                  e.failAnalysis(
                     errorClass = "_LEGACY_ERROR_TEMP_2430",
                     messageParameters = Map(
                       "operator" -> operator.nodeName,
@@ -587,10 +579,9 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
             }
 
             if (badReferences.nonEmpty) {
-              failAnalysis(
+              create.failAnalysis(
                 errorClass = "_LEGACY_ERROR_TEMP_2431",
-                messageParameters = Map(
-                  "cols" -> badReferences.mkString(", ")))
+                messageParameters = Map("cols" -> badReferences.mkString(", 
")))
             }
 
             create.tableSchema.foreach(f => 
TypeUtils.failWithIntervalType(f.dataType))
@@ -624,15 +615,14 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
               msgForMissingAttributes
             }
 
-            failAnalysis(
+            o.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2432",
               messageParameters = Map("msg" -> msg))
 
           case p @ Project(exprs, _) if containsMultipleGenerators(exprs) =>
-            failAnalysis(
+            p.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2433",
-              messageParameters = Map(
-                "sqlExprs" -> exprs.map(_.sql).mkString(",")))
+              messageParameters = Map("sqlExprs" -> 
exprs.map(_.sql).mkString(",")))
 
           case p @ Project(projectList, _) =>
             projectList.foreach(_.transformDownWithPruning(
@@ -643,7 +633,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case j: Join if !j.duplicateResolved =>
             val conflictingAttributes = 
j.left.outputSet.intersect(j.right.outputSet)
-            failAnalysis(
+            j.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2434",
               messageParameters = Map(
                 "plan" -> plan.toString,
@@ -651,7 +641,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case i: Intersect if !i.duplicateResolved =>
             val conflictingAttributes = 
i.left.outputSet.intersect(i.right.outputSet)
-            failAnalysis(
+            i.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2435",
               messageParameters = Map(
                 "plan" -> plan.toString,
@@ -659,7 +649,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case e: Except if !e.duplicateResolved =>
             val conflictingAttributes = 
e.left.outputSet.intersect(e.right.outputSet)
-            failAnalysis(
+            e.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2436",
               messageParameters = Map(
                 "plan" -> plan.toString,
@@ -667,7 +657,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
 
           case j: AsOfJoin if !j.duplicateResolved =>
             val conflictingAttributes = 
j.left.outputSet.intersect(j.right.outputSet)
-            failAnalysis(
+            j.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2437",
               messageParameters = Map(
                 "plan" -> plan.toString,
@@ -677,7 +667,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
           // used in equality comparison, remove this type check once we 
support it.
           case o if mapColumnInSetOperation(o).isDefined =>
             val mapCol = mapColumnInSetOperation(o).get
-            failAnalysis(
+            o.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2438",
               messageParameters = Map(
                 "colName" -> mapCol.name,
@@ -689,7 +679,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
             // Lateral join is checked in checkSubqueryExpression.
             !o.isInstanceOf[LateralJoin] =>
             // The rule above is used to check Aggregate operator.
-            failAnalysis(
+            o.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2439",
               messageParameters = Map(
                 "sqlExprs" -> o.expressions.map(_.sql).mkString(","),
@@ -701,7 +691,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
           case f @ Filter(condition, _)
             if PlanHelper.specialExpressionsInUnsupportedOperator(f).nonEmpty 
=>
             val invalidExprSqls = 
PlanHelper.specialExpressionsInUnsupportedOperator(f).map(_.sql)
-            failAnalysis(
+            f.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2440",
               messageParameters = Map(
                 "condition" -> condition.sql,
@@ -710,7 +700,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
           case other if 
PlanHelper.specialExpressionsInUnsupportedOperator(other).nonEmpty =>
             val invalidExprSqls =
               
PlanHelper.specialExpressionsInUnsupportedOperator(other).map(_.sql)
-            failAnalysis(
+            other.failAnalysis(
               errorClass = "_LEGACY_ERROR_TEMP_2441",
               messageParameters = Map(
                 "operator" -> other.nodeName,
@@ -723,7 +713,7 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
     extendedCheckRules.foreach(_(plan))
     plan.foreachUp {
       case o if !o.resolved =>
-        failAnalysis(
+        o.failAnalysis(
           errorClass = "_LEGACY_ERROR_TEMP_2442",
           messageParameters = Map("operator" -> 
o.simpleString(SQLConf.get.maxToStringFields)))
       case _ =>
diff --git a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out 
b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out
index ea3968fdbdf..f86d26fc92c 100644
--- a/sql/core/src/test/resources/sql-tests/results/except-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/except-all.sql.out
@@ -147,7 +147,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "ExceptAll",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 45,
+    "fragment" : "SELECT * FROM tab1\nEXCEPT ALL\nSELECT array(1)"
+  } ]
 }
 
 
@@ -227,7 +234,14 @@ org.apache.spark.sql.AnalysisException
     "nColNum" : "2",
     "nTab" : "second",
     "operator" : "ExceptAll"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 51,
+    "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out 
b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out
index 4ca429ce7bc..05277d850c3 100644
--- a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out
@@ -466,7 +466,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2445"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2445",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 22,
+    "stopIndex" : 37,
+    "fragment" : "GROUPING(course)"
+  } ]
 }
 
 
@@ -477,7 +484,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2407"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2407",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 22,
+    "stopIndex" : 46,
+    "fragment" : "GROUPING_ID(course, year)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out 
b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out
index 7d2191dfa8a..01033fa564a 100644
--- a/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/group-by-filter.sql.out
@@ -53,7 +53,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(count(testdata.b) FILTER (WHERE (testdata.a >= 2)) AS 
`count(b) FILTER (WHERE (a >= 2))`)",
     "sqlExpr" : "testdata.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 8,
+    "fragment" : "a"
+  } ]
 }
 
 
diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out 
b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out
index 578fa1588b3..6ccc0c34ff0 100644
--- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out
@@ -20,7 +20,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(count(testdata.b) AS `count(b)`)",
     "sqlExpr" : "testdata.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 8,
+    "fragment" : "a"
+  } ]
 }
 
 
@@ -209,7 +216,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2424",
   "messageParameters" : {
     "sqlExpr" : "count(testdata.b)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 15,
+    "fragment" : "COUNT(b)"
+  } ]
 }
 
 
@@ -342,7 +356,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "()",
     "sqlExpr" : "id"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 16,
+    "stopIndex" : 24,
+    "fragment" : "range(10)"
+  } ]
 }
 
 
@@ -382,7 +403,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "(max(id) > CAST(0 AS BIGINT))",
     "invalidExprSqls" : "max(id)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 42,
+    "fragment" : "SELECT 1 FROM range(10) HAVING MAX(id) > 0"
+  } ]
 }
 
 
@@ -779,7 +807,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "(count(1) > 1L)",
     "invalidExprSqls" : "count(1)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 49,
+    "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) > 1L"
+  } ]
 }
 
 
@@ -794,7 +829,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "((count(1) + 1L) > 1L)",
     "invalidExprSqls" : "count(1)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 54,
+    "fragment" : "SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L"
+  } ]
 }
 
 
@@ -809,7 +851,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "(((test_agg.k = 1) OR (test_agg.k = 2)) OR (((count(1) + 
1L) > 1L) OR (max(test_agg.k) > 1)))",
     "invalidExprSqls" : "count(1), max(test_agg.k)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 86,
+    "fragment" : "SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or 
count(*) + 1L > 1L or max(k) > 1"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out 
b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out
index 263476c911f..8eaf5cbcf59 100644
--- a/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/intersect-all.sql.out
@@ -104,7 +104,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "IntersectAll",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 51,
+    "fragment" : "SELECT * FROM tab1\nINTERSECT ALL\nSELECT array(1), 2"
+  } ]
 }
 
 
@@ -123,7 +130,14 @@ org.apache.spark.sql.AnalysisException
     "nColNum" : "2",
     "nTab" : "second",
     "operator" : "IntersectAll"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 54,
+    "fragment" : "SELECT k FROM tab1\nINTERSECT ALL\nSELECT k, v FROM tab2"
+  } ]
 }
 
 
diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out 
b/sql/core/src/test/resources/sql-tests/results/limit.sql.out
index c5320bf4869..1e21a88fe8a 100644
--- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out
@@ -55,7 +55,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "name" : "limit",
     "v" : "-1"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 30,
+    "stopIndex" : 31,
+    "fragment" : "-1"
+  } ]
 }
 
 
@@ -70,7 +77,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "name" : "limit",
     "v" : "-1"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 37,
+    "stopIndex" : 38,
+    "fragment" : "-1"
+  } ]
 }
 
 
@@ -93,7 +107,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "limitExpr" : "CAST(NULL AS INT)",
     "name" : "limit"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 30,
+    "stopIndex" : 46,
+    "fragment" : "CAST(NULL AS INT)"
+  } ]
 }
 
 
@@ -108,7 +129,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "limitExpr" : "(spark_catalog.default.testdata.key > 3)",
     "name" : "limit"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 30,
+    "stopIndex" : 36,
+    "fragment" : "key > 3"
+  } ]
 }
 
 
@@ -138,7 +166,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "dataType" : "string",
     "name" : "limit"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 30,
+    "stopIndex" : 32,
+    "fragment" : "'a'"
+  } ]
 }
 
 
diff --git a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out 
b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
index 4c5af9caf4a..38319875c71 100644
--- a/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/percentiles.sql.out
@@ -182,7 +182,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_cont"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 157,
+    "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER 
(PARTITION BY department ORDER BY salary)"
+  } ]
 }
 
 
@@ -203,7 +210,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_disc"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 157,
+    "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER 
(PARTITION BY department ORDER BY salary)"
+  } ]
 }
 
 
@@ -223,7 +237,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "median"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 119,
+    "fragment" : "median(salary) OVER (PARTITION BY department ORDER BY 
salary)"
+  } ]
 }
 
 
@@ -244,7 +265,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_cont"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 190,
+    "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER 
(PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)"
+  } ]
 }
 
 
@@ -265,7 +293,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_disc"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 190,
+    "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER 
(PARTITION BY department ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING)"
+  } ]
 }
 
 
@@ -285,7 +320,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "median"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 152,
+    "fragment" : "median(salary) OVER (PARTITION BY department ROWS BETWEEN 
UNBOUNDED PRECEDING AND 1 FOLLOWING)"
+  } ]
 }
 
 
@@ -370,7 +412,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_cont"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 117,
+    "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w"
+  } ]
 }
 
 
@@ -392,7 +441,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_disc"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 117,
+    "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w"
+  } ]
 }
 
 
@@ -413,7 +469,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "median"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 79,
+    "fragment" : "median(salary) OVER w"
+  } ]
 }
 
 
@@ -435,7 +498,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_cont"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 117,
+    "fragment" : "percentile_cont(0.25) WITHIN GROUP (ORDER BY salary) OVER w"
+  } ]
 }
 
 
@@ -457,7 +527,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "percentile_disc"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 117,
+    "fragment" : "percentile_disc(0.25) WITHIN GROUP (ORDER BY salary) OVER w"
+  } ]
 }
 
 
@@ -478,7 +555,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2411",
   "messageParameters" : {
     "aggFunc" : "median"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 59,
+    "stopIndex" : 79,
+    "fragment" : "median(salary) OVER w"
+  } ]
 }
 
 
diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out 
b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out
index bd172f9843c..b15a8652a3b 100644
--- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out
@@ -284,7 +284,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2420"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2420",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 82,
+    "stopIndex" : 94,
+    "fragment" : "avg(earnings)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out
index 027cef20596..bafa8ba1657 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part3.sql.out
@@ -6,7 +6,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2420"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2420",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 12,
+    "stopIndex" : 23,
+    "fragment" : "min(unique1)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
index 754ef794e83..f4f62be010a 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/limit.sql.out
@@ -136,7 +136,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "limitExpr" : "CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN 
CAST(NULL AS BIGINT) END",
     "name" : "limit"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 31,
+    "stopIndex" : 76,
+    "fragment" : "case when random() < 0.5 then bigint(null) end"
+  } ]
 }
 
 
@@ -151,7 +158,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "limitExpr" : "CASE WHEN (_nondeterministic < CAST(0.5BD AS DOUBLE)) THEN 
CAST(NULL AS BIGINT) END",
     "name" : "offset"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 32,
+    "stopIndex" : 77,
+    "fragment" : "case when random() < 0.5 then bigint(null) end"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out
index 909e3feb25a..ef7e8ef4e26 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_having.sql.out
@@ -145,7 +145,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(min(spark_catalog.default.test_having.a) AS `min(a#x)`, 
max(spark_catalog.default.test_having.a) AS `max(a#x)`)",
     "sqlExpr" : "spark_catalog.default.test_having.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 48,
+    "fragment" : "SELECT a FROM test_having HAVING min(a) < max(a)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
index 5138e6adab2..38005e0c607 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
@@ -336,7 +336,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "invalidExprSqls" : "row_number() OVER (ORDER BY 
spark_catalog.default.empsalary.salary ASC NULLS FIRST ROWS BETWEEN UNBOUNDED 
PRECEDING AND CURRENT ROW)",
     "operator" : "Join"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 25,
+    "stopIndex" : 84,
+    "fragment" : "INNER JOIN tenk1 ON row_number() OVER (ORDER BY salary) < 10"
+  } ]
 }
 
 
@@ -351,7 +358,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "invalidExprSqls" : "RANK() OVER (ORDER BY 1 ASC NULLS FIRST ROWS BETWEEN 
UNBOUNDED PRECEDING AND CURRENT ROW)",
     "operator" : "Aggregate"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 58,
+    "stopIndex" : 67,
+    "fragment" : "GROUP BY 1"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out
index 4d2c9b99d8a..cc4349e05f2 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/subquery/negative-cases/invalid-correlation.sql.out
@@ -48,7 +48,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(avg(t2.t2b) AS avg)",
     "sqlExpr" : "t2.t2b"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 109,
+    "stopIndex" : 111,
+    "fragment" : "t2b"
+  } ]
 }
 
 
@@ -70,7 +77,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2432",
   "messageParameters" : {
     "msg" : "Resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in 
operator !Filter t2c#x IN (list#x [t2b#x])."
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 35,
+    "stopIndex" : 298,
+    "fragment" : "SELECT   min(t2a)\n               FROM     t2\n              
 GROUP BY t2c\n               HAVING   t2c IN (SELECT   max(t3c)\n              
                  FROM     t3\n                                GROUP BY t3b\n   
                             HAVING   t3b > t2b )"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
index 260480d720e..27abeaf8596 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/widenSetOperationTypes.sql.out
@@ -94,7 +94,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -113,7 +120,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -132,7 +146,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -151,7 +172,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 88,
+    "fragment" : "SELECT cast(1 as tinyint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -242,7 +270,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 73,
+    "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -261,7 +296,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -280,7 +322,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 96,
+    "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -299,7 +348,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 89,
+    "fragment" : "SELECT cast(1 as smallint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -390,7 +446,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 68,
+    "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -409,7 +472,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 67,
+    "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast(2 as boolean) 
FROM t"
+  } ]
 }
 
 
@@ -428,7 +498,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 91,
+    "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 
09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -447,7 +524,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 84,
+    "fragment" : "SELECT cast(1 as int) FROM t UNION SELECT cast('2017-12-11 
09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -538,7 +622,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -557,7 +648,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -576,7 +674,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 94,
+    "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -595,7 +700,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 87,
+    "fragment" : "SELECT cast(1 as bigint) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -686,7 +798,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -705,7 +824,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 69,
+    "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -724,7 +850,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 93,
+    "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 
09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -743,7 +876,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 86,
+    "fragment" : "SELECT cast(1 as float) FROM t UNION SELECT cast('2017-12-11 
09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -834,7 +974,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -853,7 +1000,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -872,7 +1026,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 94,
+    "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -891,7 +1052,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 87,
+    "fragment" : "SELECT cast(1 as double) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -982,7 +1150,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 79,
+    "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT 
cast('2' as binary) FROM t"
+  } ]
 }
 
 
@@ -1001,7 +1176,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 78,
+    "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT cast(2 
as boolean) FROM t"
+  } ]
 }
 
 
@@ -1020,7 +1202,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 102,
+    "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -1039,7 +1228,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast(1 as decimal(10, 0)) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -1130,7 +1326,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -1149,7 +1352,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as string) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -1186,7 +1396,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
tinyint) FROM t"
+  } ]
 }
 
 
@@ -1205,7 +1422,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 73,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
smallint) FROM t"
+  } ]
 }
 
 
@@ -1224,7 +1448,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 68,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
int) FROM t"
+  } ]
 }
 
 
@@ -1243,7 +1474,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
bigint) FROM t"
+  } ]
 }
 
 
@@ -1262,7 +1500,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
float) FROM t"
+  } ]
 }
 
 
@@ -1281,7 +1526,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
double) FROM t"
+  } ]
 }
 
 
@@ -1300,7 +1552,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 79,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
decimal(10, 0)) FROM t"
+  } ]
 }
 
 
@@ -1319,7 +1578,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
string) FROM t"
+  } ]
 }
 
 
@@ -1347,7 +1613,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT cast(2 as 
boolean) FROM t"
+  } ]
 }
 
 
@@ -1366,7 +1639,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 96,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -1385,7 +1665,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 89,
+    "fragment" : "SELECT cast('1' as binary) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -1404,7 +1691,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 71,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
tinyint) FROM t"
+  } ]
 }
 
 
@@ -1423,7 +1717,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
smallint) FROM t"
+  } ]
 }
 
 
@@ -1442,7 +1743,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 67,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as int) 
FROM t"
+  } ]
 }
 
 
@@ -1461,7 +1769,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
bigint) FROM t"
+  } ]
 }
 
 
@@ -1480,7 +1795,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 69,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
float) FROM t"
+  } ]
 }
 
 
@@ -1499,7 +1821,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
double) FROM t"
+  } ]
 }
 
 
@@ -1518,7 +1847,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 78,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
decimal(10, 0)) FROM t"
+  } ]
 }
 
 
@@ -1537,7 +1873,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 70,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast(2 as 
string) FROM t"
+  } ]
 }
 
 
@@ -1556,7 +1899,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 72,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT cast('2' as 
binary) FROM t"
+  } ]
 }
 
 
@@ -1583,7 +1933,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT 
cast('2017-12-11 09:30:00.0' as timestamp) FROM t"
+  } ]
 }
 
 
@@ -1602,7 +1959,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 88,
+    "fragment" : "SELECT cast(1 as boolean) FROM t UNION SELECT 
cast('2017-12-11 09:30:00' as date) FROM t"
+  } ]
 }
 
 
@@ -1621,7 +1985,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as tinyint) FROM t"
+  } ]
 }
 
 
@@ -1640,7 +2011,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 96,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as smallint) FROM t"
+  } ]
 }
 
 
@@ -1659,7 +2037,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 91,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as int) FROM t"
+  } ]
 }
 
 
@@ -1678,7 +2063,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 94,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as bigint) FROM t"
+  } ]
 }
 
 
@@ -1697,7 +2089,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 93,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as float) FROM t"
+  } ]
 }
 
 
@@ -1716,7 +2115,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 94,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as double) FROM t"
+  } ]
 }
 
 
@@ -1735,7 +2141,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 102,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as decimal(10, 0)) FROM t"
+  } ]
 }
 
 
@@ -1763,7 +2176,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 96,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast('2' as binary) FROM t"
+  } ]
 }
 
 
@@ -1782,7 +2202,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00.0' as timestamp) FROM t 
UNION SELECT cast(2 as boolean) FROM t"
+  } ]
 }
 
 
@@ -1819,7 +2246,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 88,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as tinyint) FROM t"
+  } ]
 }
 
 
@@ -1838,7 +2272,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 89,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as smallint) FROM t"
+  } ]
 }
 
 
@@ -1857,7 +2298,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 84,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as int) FROM t"
+  } ]
 }
 
 
@@ -1876,7 +2324,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 87,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as bigint) FROM t"
+  } ]
 }
 
 
@@ -1895,7 +2350,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 86,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as float) FROM t"
+  } ]
 }
 
 
@@ -1914,7 +2376,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 87,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as double) FROM t"
+  } ]
 }
 
 
@@ -1933,7 +2402,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 95,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as decimal(10, 0)) FROM t"
+  } ]
 }
 
 
@@ -1961,7 +2437,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 89,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast('2' as binary) FROM t"
+  } ]
 }
 
 
@@ -1980,7 +2463,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "Union",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 88,
+    "fragment" : "SELECT cast('2017-12-12 09:30:00' as date) FROM t UNION 
SELECT cast(2 as boolean) FROM t"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out
index f9428ef8405..b696de04e9a 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by-ordinal.sql.out
@@ -391,7 +391,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "()",
     "sqlExpr" : "data.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 32,
+    "stopIndex" : 50,
+    "fragment" : "group by cube(1, 3)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out
index bc65c1d471a..a2b6b03b2b2 100644
--- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf-group-by.sql.out
@@ -20,7 +20,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "()",
     "sqlExpr" : "testdata.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 8,
+    "fragment" : "a"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out
index d1440c86f46..868f9f0c9ce 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part3.sql.out
@@ -6,7 +6,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2420"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2420",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 16,
+    "stopIndex" : 27,
+    "fragment" : "min(unique1)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out
 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out
index 3ca153579e3..54f7a538832 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_having.sql.out
@@ -145,7 +145,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(min(spark_catalog.default.test_having.a) AS `min(a#x)`, 
max(spark_catalog.default.test_having.a) AS `max(a#x)`)",
     "sqlExpr" : "spark_catalog.default.test_having.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 63,
+    "fragment" : "SELECT udf(a) FROM test_having HAVING udf(min(a)) < 
udf(max(a))"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out
index b6796be1a71..09849c2e008 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out
@@ -147,7 +147,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "ExceptAll",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 51,
+    "fragment" : "SELECT udf(c1) FROM tab1\nEXCEPT ALL\nSELECT array(1)"
+  } ]
 }
 
 
@@ -227,7 +234,14 @@ org.apache.spark.sql.AnalysisException
     "nColNum" : "2",
     "nTab" : "second",
     "operator" : "ExceptAll"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 51,
+    "fragment" : "SELECT k FROM tab3\nEXCEPT ALL\nSELECT k, v FROM tab4"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
index c78727509bb..4dbf10dd431 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out
@@ -208,7 +208,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2445"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2445",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 27,
+    "stopIndex" : 42,
+    "fragment" : "GROUPING(course)"
+  } ]
 }
 
 
@@ -219,7 +226,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2407"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2407",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 27,
+    "stopIndex" : 51,
+    "fragment" : "GROUPING_ID(course, year)"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
index b075240bb19..4d336adc412 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
@@ -20,7 +20,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "(CAST(udf(cast(count(b) as string)) AS BIGINT) AS 
`udf(count(b))`)",
     "sqlExpr" : "testdata.a"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 42,
+    "fragment" : "SELECT udf(a), udf(COUNT(b)) FROM testData"
+  } ]
 }
 
 
@@ -186,7 +193,14 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "_LEGACY_ERROR_TEMP_2424",
   "messageParameters" : {
     "sqlExpr" : "CAST(udf(cast(count(b) as string)) AS BIGINT)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 20,
+    "fragment" : "udf(COUNT(b))"
+  } ]
 }
 
 
@@ -319,7 +333,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "aggExprs" : "()",
     "sqlExpr" : "id"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 21,
+    "stopIndex" : 29,
+    "fragment" : "range(10)"
+  } ]
 }
 
 
@@ -608,7 +629,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "(count(1) > 1L)",
     "invalidExprSqls" : "count(1)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 54,
+    "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L"
+  } ]
 }
 
 
@@ -623,7 +651,14 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "((count(1) + 1L) > 1L)",
     "invalidExprSqls" : "count(1)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 59,
+    "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L"
+  } ]
 }
 
 
@@ -638,5 +673,12 @@ org.apache.spark.sql.AnalysisException
   "messageParameters" : {
     "condition" : "(((test_agg.k = 1) OR (test_agg.k = 2)) OR (((count(1) + 
1L) > 1L) OR (max(test_agg.k) > 1)))",
     "invalidExprSqls" : "count(1), max(test_agg.k)"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 91,
+    "fragment" : "SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or 
count(*) + 1L > 1L or max(k) > 1"
+  } ]
 }
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out
index a838fbc83cb..e3f62686936 100644
--- 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out
@@ -104,7 +104,14 @@ org.apache.spark.sql.AnalysisException
     "hint" : "",
     "operator" : "IntersectAll",
     "ti" : "second"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 64,
+    "fragment" : "SELECT k, udf(v) FROM tab1\nINTERSECT ALL\nSELECT array(1), 
udf(2)"
+  } ]
 }
 
 
@@ -123,7 +130,14 @@ org.apache.spark.sql.AnalysisException
     "nColNum" : "2",
     "nTab" : "second",
     "operator" : "IntersectAll"
-  }
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 1,
+    "stopIndex" : 69,
+    "fragment" : "SELECT udf(k) FROM tab1\nINTERSECT ALL\nSELECT udf(k), 
udf(v) FROM tab2"
+  } ]
 }
 
 
diff --git 
a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out 
b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out
index a48b58d5202..ca7b47efadd 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out
@@ -284,7 +284,14 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_2420"
+  "errorClass" : "_LEGACY_ERROR_TEMP_2420",
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 86,
+    "stopIndex" : 98,
+    "fragment" : "avg(earnings)"
+  } ]
 }
 
 


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

Reply via email to