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/06/18 05:39:22 UTC

[spark] branch master updated: [SPARK-39492][SQL] Rework MISSING_COLUMN

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

gengliang 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 0f2fe34d962 [SPARK-39492][SQL] Rework MISSING_COLUMN
0f2fe34d962 is described below

commit 0f2fe34d962795ee03d5f5a6667e8f5b935cef72
Author: Serge Rielau <sr...@users.noreply.github.com>
AuthorDate: Fri Jun 17 22:39:06 2022 -0700

    [SPARK-39492][SQL] Rework MISSING_COLUMN
    
    ### What changes were proposed in this pull request?
    Replace the error class MISSING_COLUMN with UNRESOLVED_COLUMN wit the following new text:
    
    A column or function parameter with name <objectName> cannot be resolved. Did you mean one of the following? [<objectList>]
    
    ### Why are the changes needed?
    
    The existing class name and text does not reflect what is really happening. The column may well exist, it may just not be within scope.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, we we replace an error class name.
    
    ### How was this patch tested?
    
    test all affected test suites with MISSING_COLUMN
    
    Closes #36891 from srielau/SPARK-39492-Rework-MISSING_COLUMN.
    
    Lead-authored-by: Serge Rielau <sr...@users.noreply.github.com>
    Co-authored-by: Serge Rielau <se...@databricks.com>
    Signed-off-by: Gengliang Wang <ge...@apache.org>
---
 core/src/main/resources/error/error-classes.json   | 12 ++++----
 .../org/apache/spark/SparkThrowableSuite.scala     |  7 +++--
 python/pyspark/pandas/tests/test_indexops_spark.py |  4 +--
 python/pyspark/sql/tests/test_utils.py             |  4 +--
 .../spark/sql/catalyst/analysis/Analyzer.scala     |  7 ++---
 .../sql/catalyst/analysis/CheckAnalysis.scala      |  5 ++--
 .../spark/sql/errors/QueryCompilationErrors.scala  |  9 ++++++
 .../apache/spark/sql/errors/QueryErrorsBase.scala  |  3 +-
 .../sql/catalyst/analysis/AnalysisErrorSuite.scala | 19 +++++++------
 .../sql/catalyst/analysis/AnalysisSuite.scala      | 12 ++++----
 .../catalyst/analysis/ResolveSubquerySuite.scala   | 20 +++++++-------
 .../catalyst/analysis/V2WriteAnalysisSuite.scala   |  4 +--
 .../results/columnresolution-negative.sql.out      |  8 +++---
 .../resources/sql-tests/results/group-by.sql.out   |  4 +--
 .../sql-tests/results/join-lateral.sql.out         |  8 +++---
 .../sql-tests/results/natural-join.sql.out         |  2 +-
 .../test/resources/sql-tests/results/pivot.sql.out |  4 +--
 .../results/postgreSQL/aggregates_part1.sql.out    |  2 +-
 .../results/postgreSQL/create_view.sql.out         |  2 +-
 .../sql-tests/results/postgreSQL/join.sql.out      | 16 +++++------
 .../results/postgreSQL/select_having.sql.out       |  2 +-
 .../results/postgreSQL/select_implicit.sql.out     |  4 +--
 .../sql-tests/results/postgreSQL/union.sql.out     |  2 +-
 .../sql-tests/results/query_regex_column.sql.out   | 16 +++++------
 .../negative-cases/invalid-correlation.sql.out     |  2 +-
 .../sql-tests/results/table-aliases.sql.out        |  2 +-
 .../udf/postgreSQL/udf-aggregates_part1.sql.out    |  2 +-
 .../results/udf/postgreSQL/udf-join.sql.out        | 16 +++++------
 .../udf/postgreSQL/udf-select_having.sql.out       |  2 +-
 .../udf/postgreSQL/udf-select_implicit.sql.out     |  4 +--
 .../sql-tests/results/udf/udf-group-by.sql.out     |  2 +-
 .../sql-tests/results/udf/udf-pivot.sql.out        |  4 +--
 .../apache/spark/sql/DataFrameFunctionsSuite.scala | 32 +++++++++++-----------
 .../apache/spark/sql/DataFrameSelfJoinSuite.scala  |  5 +++-
 .../org/apache/spark/sql/DataFrameSuite.scala      |  4 +--
 .../spark/sql/DataFrameWindowFunctionsSuite.scala  |  4 +--
 .../scala/org/apache/spark/sql/DatasetSuite.scala  |  8 +++---
 .../org/apache/spark/sql/SQLInsertTestSuite.scala  |  4 +--
 .../scala/org/apache/spark/sql/SubquerySuite.scala |  4 +--
 .../test/scala/org/apache/spark/sql/UDFSuite.scala |  4 +--
 .../spark/sql/connector/DataSourceV2SQLSuite.scala | 26 +++++++++---------
 .../sql/errors/QueryCompilationErrorsSuite.scala   | 17 ++++++------
 .../apache/spark/sql/execution/SQLViewSuite.scala  | 14 +++++++---
 .../sql/execution/datasources/csv/CSVSuite.scala   |  4 +--
 .../sql/execution/datasources/json/JsonSuite.scala |  4 +--
 .../org/apache/spark/sql/sources/InsertSuite.scala |  9 +++---
 .../apache/spark/sql/hive/HiveParquetSuite.scala   |  4 +--
 47 files changed, 187 insertions(+), 166 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index 275566589f5..d4c0910c5ad 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -251,12 +251,6 @@
       "Key <keyValue> does not exist. Use `try_element_at` to tolerate non-existent key and return NULL instead. If necessary set <config> to \"false\" to bypass this error."
     ]
   },
-  "MISSING_COLUMN" : {
-    "message" : [
-      "Column '<columnName>' does not exist. Did you mean one of the following? [<proposal>]"
-    ],
-    "sqlState" : "42000"
-  },
   "MISSING_STATIC_PARTITION_COLUMN" : {
     "message" : [
       "Unknown static partition column: <columnName>"
@@ -352,6 +346,12 @@
     ],
     "sqlState" : "42000"
   },
+  "UNRESOLVED_COLUMN" : {
+    "message" : [
+      "A column or function parameter with name <objectName> cannot be resolved. Did you mean one of the following? [<objectList>]"
+    ],
+    "sqlState" : "42000"
+  },
   "UNSUPPORTED_DATATYPE" : {
     "message" : [
       "Unsupported data type <typeName>"
diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
index 91019973dea..4b14e2402a7 100644
--- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala
@@ -160,7 +160,7 @@ class SparkThrowableSuite extends SparkFunSuite {
   test("Check if message parameters match message format") {
     // Requires 2 args
     intercept[IllegalFormatException] {
-      getMessage("MISSING_COLUMN", null, Array.empty)
+      getMessage("UNRESOLVED_COLUMN", null, Array.empty)
     }
 
     // Does not fail with too many args (expects 0 args)
@@ -172,8 +172,9 @@ class SparkThrowableSuite extends SparkFunSuite {
   }
 
   test("Error message is formatted") {
-    assert(getMessage("MISSING_COLUMN", null, Array("foo", "bar, baz")) ==
-      "[MISSING_COLUMN] Column 'foo' does not exist. Did you mean one of the following? [bar, baz]")
+    assert(getMessage("UNRESOLVED_COLUMN", null, Array("`foo`", "`bar`, `baz`")) ==
+      "[UNRESOLVED_COLUMN] A column or function parameter with name `foo` cannot be resolved. " +
+        "Did you mean one of the following? [`bar`, `baz`]")
   }
 
   test("Try catching legacy SparkError") {
diff --git a/python/pyspark/pandas/tests/test_indexops_spark.py b/python/pyspark/pandas/tests/test_indexops_spark.py
index 88bf395b047..275ef77f71b 100644
--- a/python/pyspark/pandas/tests/test_indexops_spark.py
+++ b/python/pyspark/pandas/tests/test_indexops_spark.py
@@ -39,7 +39,7 @@ class SparkIndexOpsMethodsTest(PandasOnSparkTestCase, SQLTestUtils):
         ):
             self.psser.spark.transform(lambda scol: 1)
 
-        with self.assertRaisesRegex(AnalysisException, "Column.*non-existent.*does not exist"):
+        with self.assertRaisesRegex(AnalysisException, ".*UNRESOLVED_COLUMN.*`non-existent`.*"):
             self.psser.spark.transform(lambda scol: F.col("non-existent"))
 
     def test_multiindex_transform_negative(self):
@@ -59,7 +59,7 @@ class SparkIndexOpsMethodsTest(PandasOnSparkTestCase, SQLTestUtils):
         ):
             self.psser.spark.apply(lambda scol: 1)
 
-        with self.assertRaisesRegex(AnalysisException, "Column.*non-existent.*does not exist"):
+        with self.assertRaisesRegex(AnalysisException, ".*UNRESOLVED_COLUMN.*`non-existent`.*"):
             self.psser.spark.transform(lambda scol: F.col("non-existent"))
 
 
diff --git a/python/pyspark/sql/tests/test_utils.py b/python/pyspark/sql/tests/test_utils.py
index f69fefc24eb..6f504583af7 100644
--- a/python/pyspark/sql/tests/test_utils.py
+++ b/python/pyspark/sql/tests/test_utils.py
@@ -36,7 +36,7 @@ class UtilsTests(ReusedSQLTestCase):
         try:
             self.spark.sql("select `中文字段`")
         except AnalysisException as e:
-            self.assertRegex(str(e), "Column '`中文字段`' does not exist")
+            self.assertRegex(str(e), ".*UNRESOLVED_COLUMN.*`中文字段`.*")
 
     def test_spark_upgrade_exception(self):
         # SPARK-32161 : Test case to Handle SparkUpgradeException in pythonic way
@@ -72,7 +72,7 @@ class UtilsTests(ReusedSQLTestCase):
         try:
             self.spark.sql("""SELECT a""")
         except AnalysisException as e:
-            self.assertEquals(e.getErrorClass(), "MISSING_COLUMN")
+            self.assertEquals(e.getErrorClass(), "UNRESOLVED_COLUMN")
             self.assertEquals(e.getSqlState(), "42000")
 
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 9fe9d490539..931a0fcf77f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -3419,10 +3419,9 @@ class Analyzer(override val catalogManager: CatalogManager)
         i.userSpecifiedCols, "in the column list", resolver)
 
       i.userSpecifiedCols.map { col =>
-        i.table.resolve(Seq(col), resolver)
-          .getOrElse(i.failAnalysis(
-            errorClass = "MISSING_COLUMN",
-            messageParameters = Array(col, i.table.output.map(_.name).mkString(", "))))
+        i.table.resolve(Seq(col), resolver).getOrElse(
+          throw QueryCompilationErrors.unresolvedColumnError(
+            col, i.table.output.map(_.name), i.origin))
       }
     }
 
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 416e3a2b834..f9f8b590a31 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
@@ -179,9 +179,8 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog {
             val missingCol = a.sql
             val candidates = operator.inputSet.toSeq.map(_.qualifiedName)
             val orderedCandidates = StringUtils.orderStringsBySimilarity(missingCol, candidates)
-            a.failAnalysis(
-              errorClass = "MISSING_COLUMN",
-              messageParameters = Array(missingCol, orderedCandidates.mkString(", ")))
+            throw QueryCompilationErrors.unresolvedColumnError(
+              missingCol, orderedCandidates, a.origin)
 
           case s: Star =>
             withPosition(s) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 2a8692efd0d..4ee53c56f69 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -144,6 +144,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
         s"side of the join. The $side-side columns: [${plan.output.map(_.name).mkString(", ")}]")
   }
 
+  def unresolvedColumnError(
+      colName: String, candidates: Seq[String], origin: Origin): Throwable = {
+    val candidateIds = candidates.map(candidate => toSQLId(candidate))
+    new AnalysisException(
+      errorClass = "UNRESOLVED_COLUMN",
+      messageParameters = Array(toSQLId(colName), candidateIds.mkString(", ")),
+      origin = origin)
+  }
+
   def dataTypeMismatchForDeserializerError(
       dataType: DataType, desiredType: String): Throwable = {
     new AnalysisException(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala
index 558401cb4e9..345fc8e0232 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryErrorsBase.scala
@@ -19,6 +19,7 @@ package org.apache.spark.sql.errors
 
 import java.util.Locale
 
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
 import org.apache.spark.sql.catalyst.expressions.{Expression, Literal}
 import org.apache.spark.sql.catalyst.util.{quoteIdentifier, toPrettySQL}
 import org.apache.spark.sql.types.{DataType, DoubleType, FloatType}
@@ -72,7 +73,7 @@ private[sql] trait QueryErrorsBase {
   }
 
   def toSQLId(parts: String): String = {
-    toSQLId(parts.split("\\."))
+    toSQLId(UnresolvedAttribute.parseAttributeName(parts))
   }
 
   def toSQLType(t: DataType): String = {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
index 58cc4418ab0..4ac665f9d87 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala
@@ -297,16 +297,16 @@ class AnalysisErrorSuite extends AnalysisTest {
   errorClassTest(
     "unresolved attributes",
     testRelation.select($"abcd"),
-    "MISSING_COLUMN",
-    Array("abcd", "a"))
+    "UNRESOLVED_COLUMN",
+    Array("`abcd`", "`a`"))
 
   errorClassTest(
     "unresolved attributes with a generated name",
     testRelation2.groupBy($"a")(max($"b"))
       .where(sum($"b") > 0)
       .orderBy($"havingCondition".asc),
-    "MISSING_COLUMN",
-    Array("havingCondition", "max(b)"))
+    "UNRESOLVED_COLUMN",
+    Array("`havingCondition`", "`max(b)`"))
 
   errorTest(
     "unresolved star expansion in max",
@@ -321,8 +321,8 @@ class AnalysisErrorSuite extends AnalysisTest {
   errorClassTest(
     "sorting by attributes are not from grouping expressions",
     testRelation2.groupBy($"a", $"c")($"a", $"c", count($"a").as("a3")).orderBy($"b".asc),
-    "MISSING_COLUMN",
-    Array("b", "a, c, a3"))
+    "UNRESOLVED_COLUMN",
+    Array("`b`", "`a`, `c`, `a3`"))
 
   errorTest(
     "non-boolean filters",
@@ -415,8 +415,8 @@ class AnalysisErrorSuite extends AnalysisTest {
     "SPARK-9955: correct error message for aggregate",
     // When parse SQL string, we will wrap aggregate expressions with UnresolvedAlias.
     testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))),
-    "MISSING_COLUMN",
-    Array("bad_column", "a, b, c, d, e"))
+    "UNRESOLVED_COLUMN",
+    Array("`bad_column`", "`a`, `b`, `c`, `d`, `e`"))
 
   errorTest(
     "slide duration greater than window in time window",
@@ -836,7 +836,8 @@ class AnalysisErrorSuite extends AnalysisTest {
   errorTest(
     "SPARK-34920: error code to error message",
     testRelation2.where($"bad_column" > 1).groupBy($"a")(UnresolvedAlias(max($"b"))),
-    "Column 'bad_column' does not exist. Did you mean one of the following? [a, b, c, d, e]"
+    "[UNRESOLVED_COLUMN] A column or function parameter with name `bad_column` cannot be " +
+      "resolved. Did you mean one of the following? [`a`, `b`, `c`, `d`, `e`]"
       :: Nil)
 
   test("SPARK-35080: Unsupported correlated equality predicates in subquery") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 5c3f4b5f558..1b397935a89 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -103,8 +103,8 @@ class AnalysisSuite extends AnalysisTest with Matchers {
     assertAnalysisErrorClass(
       Project(Seq(UnresolvedAttribute("tBl.a")),
         SubqueryAlias("TbL", UnresolvedRelation(TableIdentifier("TaBlE")))),
-      "MISSING_COLUMN",
-      Array("tBl.a", "TbL.a"))
+      "UNRESOLVED_COLUMN",
+      Array("`tBl`.`a`", "`TbL`.`a`"))
 
     checkAnalysisWithoutViewWrapper(
       Project(Seq(UnresolvedAttribute("TbL.a")),
@@ -711,8 +711,8 @@ class AnalysisSuite extends AnalysisTest with Matchers {
 
   test("CTE with non-existing column alias") {
     assertAnalysisErrorClass(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"),
-      "MISSING_COLUMN",
-      Array("y", "t.x"))
+      "UNRESOLVED_COLUMN",
+      Array("`y`", "`t`.`x`"))
   }
 
   test("CTE with non-matching column alias") {
@@ -1149,8 +1149,8 @@ class AnalysisSuite extends AnalysisTest with Matchers {
         |GROUP BY c.x
         |ORDER BY c.x + c.y
         |""".stripMargin),
-      "MISSING_COLUMN",
-      Array("c.y", "x"))
+      "UNRESOLVED_COLUMN",
+      Array("`c`.`y`", "`x`"))
   }
 
   test("SPARK-38118: Func(wrong_type) in the HAVING clause should throw data mismatch error") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala
index e94c32f9b2f..11dcae12406 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveSubquerySuite.scala
@@ -133,34 +133,34 @@ class ResolveSubquerySuite extends AnalysisTest {
     // TODO: support accessing columns from outer outer query.
     assertAnalysisErrorClass(
       lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b", $"c"))),
-      "MISSING_COLUMN",
-      Array("a", ""))
+      "UNRESOLVED_COLUMN",
+      Array("`a`", ""))
   }
 
   test("lateral subquery with unresolvable attributes") {
     // SELECT * FROM t1, LATERAL (SELECT a, c)
     assertAnalysisErrorClass(
       lateralJoin(t1, t0.select($"a", $"c")),
-      "MISSING_COLUMN",
-      Array("c", "")
+      "UNRESOLVED_COLUMN",
+      Array("`c`", "")
     )
     // SELECT * FROM t1, LATERAL (SELECT a, b, c, d FROM t2)
     assertAnalysisErrorClass(
       lateralJoin(t1, t2.select($"a", $"b", $"c", $"d")),
-      "MISSING_COLUMN",
-      Array("d", "b, c")
+      "UNRESOLVED_COLUMN",
+      Array("`d`", "`b`, `c`")
     )
     // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.a))
     assertAnalysisErrorClass(
       lateralJoin(t1, lateralJoin(t2, t0.select($"t1.a"))),
-      "MISSING_COLUMN",
-      Array("t1.a", "")
+      "UNRESOLVED_COLUMN",
+      Array("`t1`.`a`", "")
     )
     // SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT a, b))
     assertAnalysisErrorClass(
       lateralJoin(t1, lateralJoin(t2, t0.select($"a", $"b"))),
-      "MISSING_COLUMN",
-      Array("a", "")
+      "UNRESOLVED_COLUMN",
+      Array("`a`", "")
     )
   }
 
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala
index 1df402d958e..7fbedc7b312 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/V2WriteAnalysisSuite.scala
@@ -688,7 +688,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest {
       LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d)))
 
     assertNotResolved(parsedPlan)
-    assertAnalysisErrorClass(parsedPlan, "MISSING_COLUMN", Array("a", "x, y"))
+    assertAnalysisErrorClass(parsedPlan, "UNRESOLVED_COLUMN", Array("`a`", "`x`, `y`"))
 
     val tableAcceptAnySchema = TestRelationAcceptAnySchema(StructType(Seq(
       StructField("x", DoubleType, nullable = false),
@@ -697,7 +697,7 @@ abstract class V2WriteAnalysisSuiteBase extends AnalysisTest {
     val parsedPlan2 = OverwriteByExpression.byPosition(tableAcceptAnySchema, query,
       LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d)))
     assertNotResolved(parsedPlan2)
-    assertAnalysisErrorClass(parsedPlan2, "MISSING_COLUMN", Array("a", "x, y"))
+    assertAnalysisErrorClass(parsedPlan2, "UNRESOLVED_COLUMN", Array("`a`", "`x`, `y`"))
   }
 
   test("SPARK-36498: reorder inner fields with byName mode") {
diff --git a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out
index 0dc27fdca15..9d0b9e44468 100644
--- a/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/columnresolution-negative.sql.out
@@ -158,7 +158,7 @@ SELECT db1.t1.i1 FROM t1, mydb2.t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'db1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1, spark_catalog.mydb2.t1.i1]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `db1`.`t1`.`i1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb2`.`t1`.`i1`, `spark_catalog`.`mydb2`.`t1`.`i1`]; line 1 pos 7
 
 
 -- !query
@@ -183,7 +183,7 @@ SELECT mydb1.t1 FROM t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'mydb1.t1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `mydb1`.`t1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb1`.`t1`.`i1`]; line 1 pos 7
 
 
 -- !query
@@ -201,7 +201,7 @@ SELECT t1 FROM mydb1.t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1' does not exist. Did you mean one of the following? [spark_catalog.mydb1.t1.i1]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb1`.`t1`.`i1`]; line 1 pos 7
 
 
 -- !query
@@ -218,7 +218,7 @@ SELECT mydb1.t1.i1 FROM t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'mydb1.t1.i1' does not exist. Did you mean one of the following? [spark_catalog.mydb2.t1.i1]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `mydb1`.`t1`.`i1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`mydb2`.`t1`.`i1`]; line 1 pos 7
 
 
 -- !query
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 09876a057c9..b84649946eb 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
@@ -164,7 +164,7 @@ SELECT a AS k, COUNT(non_existing) FROM testData GROUP BY k
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'non_existing' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 21
+[UNRESOLVED_COLUMN] A column or function parameter with name `non_existing` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 21
 
 
 -- !query
@@ -208,7 +208,7 @@ SELECT a AS k, COUNT(b) FROM testData GROUP BY k
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 47
+[UNRESOLVED_COLUMN] A column or function parameter with name `k` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 47
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out
index 63446155938..a4df85b4a49 100644
--- a/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/join-lateral.sql.out
@@ -266,7 +266,7 @@ SELECT * FROM t1 JOIN LATERAL (SELECT t1.c1 AS a, t2.c1 AS b) s JOIN t2 ON s.b =
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't2.c1' does not exist. Did you mean one of the following? []; line 1 pos 50
+[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`c1` cannot be resolved. Did you mean one of the following? []; line 1 pos 50
 
 
 -- !query
@@ -384,7 +384,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT t1.c1 + t2.c1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? []; line 1 pos 61
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`c1` cannot be resolved. Did you mean one of the following? []; line 1 pos 61
 
 
 -- !query
@@ -393,7 +393,7 @@ SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1), LATERAL (SELECT c2))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'c2' does not exist. Did you mean one of the following? []; line 1 pos 70
+[UNRESOLVED_COLUMN] A column or function parameter with name `c2` cannot be resolved. Did you mean one of the following? []; line 1 pos 70
 
 
 -- !query
@@ -420,7 +420,7 @@ SELECT * FROM t1, LATERAL (SELECT c1, (SELECT SUM(c2) FROM t2 WHERE c1 = t1.c1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1.c1' does not exist. Did you mean one of the following? [spark_catalog.default.t2.c1, spark_catalog.default.t2.c2]; line 1 pos 73
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`c1` cannot be resolved. Did you mean one of the following? [`spark_catalog`.`default`.`t2`.`c1`, `spark_catalog`.`default`.`t2`.`c2`]; line 1 pos 73
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out
index fc840486daf..83afea9d5db 100644
--- a/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/natural-join.sql.out
@@ -229,7 +229,7 @@ SELECT nt2.k FROM (SELECT * FROM nt1 natural join nt2)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'nt2.k' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.k, __auto_generated_subquery_name.v1, __auto_generated_subquery_name.v2]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `nt2`.`k` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`k`, `__auto_generated_subquery_name`.`v1`, `__auto_generated_subquery_name`.`v2`]; line 1 pos 7
 
 
 -- !query
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 613ced1dce2..f739131c627 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
@@ -229,7 +229,7 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0
+[UNRESOLVED_COLUMN] A column or function parameter with name `year` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`]; line 4 pos 0
 
 
 -- !query
@@ -323,7 +323,7 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15
+[UNRESOLVED_COLUMN] A column or function parameter with name `s` cannot be resolved. Did you mean one of the following? [`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`]; line 4 pos 15
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out
index eeffa07e11e..a0978af3606 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/aggregates_part1.sql.out
@@ -495,4 +495,4 @@ from tenk1 o
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 63
+[UNRESOLVED_COLUMN] A column or function parameter with name `o`.`unique1` cannot be resolved. Did you mean one of the following? [`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`, `i`.`stringu1`, `i`.`ten`, `i`.`odd`, `i`.`string4`, `i`.`stringu2`, `i`.`tenthous`, `i`.`twenty`, `i`.`two`, `i`.`thousand`, `i`.`fivethous`, `i`.`twothousand`]; line 2 pos 63
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out
index cb338bbc4cd..f9a02dbd41e 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/create_view.sql.out
@@ -63,7 +63,7 @@ CREATE VIEW key_dependent_view_no_cols AS
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'FROM' does not exist. Did you mean one of the following? []; line 2 pos 10
+[UNRESOLVED_COLUMN] A column or function parameter with name `FROM` cannot be resolved. Did you mean one of the following? []; line 2 pos 10
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out
index 9228558c87a..465f1d3b041 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/join.sql.out
@@ -3245,7 +3245,7 @@ select * from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 63
+[UNRESOLVED_COLUMN] A column or function parameter with name `y`.`f1` cannot be resolved. Did you mean one of the following? [`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`]; line 2 pos 63
 
 
 -- !query
@@ -3264,7 +3264,7 @@ select t1.uunique1 from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`stringu1`, `t1`.`even`, `t1`.`four`, `t1`.`string4`, `t2`.`stringu1`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`even`, `t2`.`four`, `t1`.`odd`, `t2`.`string4`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two [...]
 
 
 -- !query
@@ -3274,7 +3274,7 @@ select t2.uunique1 from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`uunique1` cannot be resolved. Did you mean one of the following? [`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`, `t1`.`hundred`, `t2`.`stringu1`, `t2`.`even`, `t2`.`four`, `t2`.`string4`, `t1`.`stringu1`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`even`, `t1`.`four`, `t2`.`odd`, `t1`.`string4`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two [...]
 
 
 -- !query
@@ -3284,7 +3284,7 @@ select uunique1 from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`, `t2`.`even`, `t1`.`four`, `t2`.`four`, `t1`.`ten`, `t2`.`ten`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`odd`, `t2`.`odd`, `t1`.`two`, `t2`.`two`, `t1`.`stringu1`, `t2`.`stringu1`, `t1`.`twenty`, `t2`.`twenty`, `t1`.`string4`, `t2`.`string4`, `t1`.`stringu2`, `t2`.`stringu2`, `t1`.`tenthous [...]
 
 
 -- !query
@@ -3484,7 +3484,7 @@ select f1,g from int4_tbl a, (select f1 as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 37
+[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 37
 
 
 -- !query
@@ -3493,7 +3493,7 @@ select f1,g from int4_tbl a, (select a.f1 as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 37
+[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 37
 
 
 -- !query
@@ -3502,7 +3502,7 @@ select f1,g from int4_tbl a cross join (select f1 as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 47
+[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 47
 
 
 -- !query
@@ -3511,7 +3511,7 @@ select f1,g from int4_tbl a cross join (select a.f1 as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 47
+[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 47
 
 
 -- !query
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 11b278aa42a..4b8945033df 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
@@ -149,7 +149,7 @@ SELECT 1 AS one FROM test_having HAVING a > 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 40
+[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`one`]; line 1 pos 40
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out
index 41147e2b779..eea7c3de203 100755
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/select_implicit.sql.out
@@ -119,7 +119,7 @@ SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(1)]; line 1 pos 61
+[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`count(1)`]; line 1 pos 61
 
 
 -- !query
@@ -324,7 +324,7 @@ SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [count(a)]; line 1 pos 61
+[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`count(a)`]; line 1 pos 61
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out
index 24ebd227e0f..18bc925cefc 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/union.sql.out
@@ -565,7 +565,7 @@ SELECT q1 FROM int8_tbl EXCEPT SELECT q2 FROM int8_tbl ORDER BY q2 LIMIT 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'q2' does not exist. Did you mean one of the following? [int8_tbl.q1]; line 1 pos 64
+[UNRESOLVED_COLUMN] A column or function parameter with name `q2` cannot be resolved. Did you mean one of the following? [`int8_tbl`.`q1`]; line 1 pos 64
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out
index f46bfd6c885..e454dc5e041 100644
--- a/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/query_regex_column.sql.out
@@ -33,7 +33,7 @@ SELECT `(a)?+.+` FROM testData2 WHERE a = 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7
 
 
 -- !query
@@ -42,7 +42,7 @@ SELECT t.`(a)?+.+` FROM testData2 t WHERE a = 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't.`(a)?+.+`' does not exist. Did you mean one of the following? [t.A, t.B, t.c, t.d]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `t`.`(a)?+.+` cannot be resolved. Did you mean one of the following? [`t`.`A`, `t`.`B`, `t`.`c`, `t`.`d`]; line 1 pos 7
 
 
 -- !query
@@ -51,7 +51,7 @@ SELECT `(a|b)` FROM testData2 WHERE a = 2
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a|b)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7
 
 
 -- !query
@@ -60,7 +60,7 @@ SELECT `(a|b)?+.+` FROM testData2 WHERE a = 2
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 7
 
 
 -- !query
@@ -69,7 +69,7 @@ SELECT SUM(`(a|b)?+.+`) FROM testData2
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a|b)?+.+`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a|b)?+.+` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 11
 
 
 -- !query
@@ -78,7 +78,7 @@ SELECT SUM(`(a)`) FROM testData2
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata2.A, testdata2.B, testdata2.c, testdata2.d]; line 1 pos 11
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a)` cannot be resolved. Did you mean one of the following? [`testdata2`.`A`, `testdata2`.`B`, `testdata2`.`c`, `testdata2`.`d`]; line 1 pos 11
 
 
 -- !query
@@ -298,7 +298,7 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)`
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a)`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a)` cannot be resolved. Did you mean one of the following? [`testdata3`.`a`, `testdata3`.`b`]; line 1 pos 38
 
 
 -- !query
@@ -307,4 +307,4 @@ SELECT SUM(a) FROM testdata3 GROUP BY `(a)?+.+`
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column '`(a)?+.+`' does not exist. Did you mean one of the following? [testdata3.a, testdata3.b]; line 1 pos 38
+[UNRESOLVED_COLUMN] A column or function parameter with name `(a)?+.+` cannot be resolved. Did you mean one of the following? [`testdata3`.`a`, `testdata3`.`b`]; line 1 pos 38
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 cf6b57efa2f..b358fed2663 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
@@ -134,4 +134,4 @@ ON     EXISTS (SELECT 1 FROM t2 WHERE t2a > t1a)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1a' does not exist. Did you mean one of the following? [t2.t2a, t2.t2b, t2.t2c]; line 4 pos 44
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1a` cannot be resolved. Did you mean one of the following? [`t2`.`t2a`, `t2`.`t2b`, `t2`.`t2c`]; line 4 pos 44
diff --git a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out
index c72e5a309bb..5aaddbf91c2 100644
--- a/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/table-aliases.sql.out
@@ -57,7 +57,7 @@ SELECT a AS col1, b AS col2 FROM testData AS t(c, d)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [t.c, t.d]; line 1 pos 7
+[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`t`.`c`, `t`.`d`]; line 1 pos 7
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out
index 6d9ab399aca..c646e999354 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-aggregates_part1.sql.out
@@ -486,4 +486,4 @@ from tenk1 o
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'o.unique1' does not exist. Did you mean one of the following? [i.unique1, i.unique2, i.hundred, i.even, i.four, i.stringu1, i.ten, i.odd, i.string4, i.stringu2, i.tenthous, i.twenty, i.two, i.thousand, i.fivethous, i.twothousand]; line 2 pos 67
+[UNRESOLVED_COLUMN] A column or function parameter with name `o`.`unique1` cannot be resolved. Did you mean one of the following? [`i`.`unique1`, `i`.`unique2`, `i`.`hundred`, `i`.`even`, `i`.`four`, `i`.`stringu1`, `i`.`ten`, `i`.`odd`, `i`.`string4`, `i`.`stringu2`, `i`.`tenthous`, `i`.`twenty`, `i`.`two`, `i`.`thousand`, `i`.`fivethous`, `i`.`twothousand`]; line 2 pos 67
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out
index 870a1537971..b95cbc00f35 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-join.sql.out
@@ -3273,7 +3273,7 @@ select * from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'y.f1' does not exist. Did you mean one of the following? [j.f1, j.f1, x.q1, x.q2]; line 2 pos 72
+[UNRESOLVED_COLUMN] A column or function parameter with name `y`.`f1` cannot be resolved. Did you mean one of the following? [`j`.`f1`, `j`.`f1`, `x`.`q1`, `x`.`q2`]; line 2 pos 72
 
 
 -- !query
@@ -3292,7 +3292,7 @@ select udf(t1.uunique1) from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't1.uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.hundred, t2.hundred, t1.stringu1, t1.even, t1.four, t1.string4, t2.stringu1, t1.stringu2, t1.ten, t1.tenthous, t2.even, t2.four, t1.odd, t2.string4, t2.stringu2, t2.ten, t2.tenthous, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.odd, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11
+[UNRESOLVED_COLUMN] A column or function parameter with name `t1`.`uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`stringu1`, `t1`.`even`, `t1`.`four`, `t1`.`string4`, `t2`.`stringu1`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`even`, `t2`.`four`, `t1`.`odd`, `t2`.`string4`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`thousand`, `t1`.`twenty`, `t1`.`two [...]
 
 
 -- !query
@@ -3302,7 +3302,7 @@ select udf(udf(t2.uunique1)) from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 't2.uunique1' does not exist. Did you mean one of the following? [t2.unique1, t1.unique1, t2.unique2, t1.unique2, t2.hundred, t1.hundred, t2.stringu1, t2.even, t2.four, t2.string4, t1.stringu1, t2.stringu2, t2.ten, t2.tenthous, t1.even, t1.four, t2.odd, t1.string4, t1.stringu2, t1.ten, t1.tenthous, t2.thousand, t2.twenty, t2.two, t2.fivethous, t1.odd, t1.thousand, t1.twenty, t1.two, t1.fivethous, t2.twothousand, t1.twothousand]; line 1 pos 15
+[UNRESOLVED_COLUMN] A column or function parameter with name `t2`.`uunique1` cannot be resolved. Did you mean one of the following? [`t2`.`unique1`, `t1`.`unique1`, `t2`.`unique2`, `t1`.`unique2`, `t2`.`hundred`, `t1`.`hundred`, `t2`.`stringu1`, `t2`.`even`, `t2`.`four`, `t2`.`string4`, `t1`.`stringu1`, `t2`.`stringu2`, `t2`.`ten`, `t2`.`tenthous`, `t1`.`even`, `t1`.`four`, `t2`.`odd`, `t1`.`string4`, `t1`.`stringu2`, `t1`.`ten`, `t1`.`tenthous`, `t2`.`thousand`, `t2`.`twenty`, `t2`.`two [...]
 
 
 -- !query
@@ -3312,7 +3312,7 @@ select udf(uunique1) from
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'uunique1' does not exist. Did you mean one of the following? [t1.unique1, t2.unique1, t1.unique2, t2.unique2, t1.even, t2.even, t1.four, t2.four, t1.ten, t2.ten, t1.hundred, t2.hundred, t1.odd, t2.odd, t1.two, t2.two, t1.stringu1, t2.stringu1, t1.twenty, t2.twenty, t1.string4, t2.string4, t1.stringu2, t2.stringu2, t1.tenthous, t2.tenthous, t1.thousand, t2.thousand, t1.fivethous, t2.fivethous, t1.twothousand, t2.twothousand]; line 1 pos 11
+[UNRESOLVED_COLUMN] A column or function parameter with name `uunique1` cannot be resolved. Did you mean one of the following? [`t1`.`unique1`, `t2`.`unique1`, `t1`.`unique2`, `t2`.`unique2`, `t1`.`even`, `t2`.`even`, `t1`.`four`, `t2`.`four`, `t1`.`ten`, `t2`.`ten`, `t1`.`hundred`, `t2`.`hundred`, `t1`.`odd`, `t2`.`odd`, `t1`.`two`, `t2`.`two`, `t1`.`stringu1`, `t2`.`stringu1`, `t1`.`twenty`, `t2`.`twenty`, `t1`.`string4`, `t2`.`string4`, `t1`.`stringu2`, `t2`.`stringu2`, `t1`.`tenthous [...]
 
 
 -- !query
@@ -3512,7 +3512,7 @@ select udf(udf(f1,g)) from int4_tbl a, (select udf(udf(f1)) as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 55
+[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 55
 
 
 -- !query
@@ -3521,7 +3521,7 @@ select udf(f1,g) from int4_tbl a, (select a.f1 as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 42
+[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 42
 
 
 -- !query
@@ -3530,7 +3530,7 @@ select udf(udf(f1,g)) from int4_tbl a cross join (select udf(f1) as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'f1' does not exist. Did you mean one of the following? []; line 1 pos 61
+[UNRESOLVED_COLUMN] A column or function parameter with name `f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 61
 
 
 -- !query
@@ -3539,7 +3539,7 @@ select udf(f1,g) from int4_tbl a cross join (select udf(udf(a.f1)) as g) ss
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a.f1' does not exist. Did you mean one of the following? []; line 1 pos 60
+[UNRESOLVED_COLUMN] A column or function parameter with name `a`.`f1` cannot be resolved. Did you mean one of the following? []; line 1 pos 60
 
 
 -- !query
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 3308be92072..5e7a9b96a82 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
@@ -149,7 +149,7 @@ SELECT 1 AS one FROM test_having HAVING udf(a) > 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'a' does not exist. Did you mean one of the following? [one]; line 1 pos 44
+[UNRESOLVED_COLUMN] A column or function parameter with name `a` cannot be resolved. Did you mean one of the following? [`one`]; line 1 pos 44
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out
index 305d099cc4c..13295adab15 100755
--- a/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/postgreSQL/udf-select_implicit.sql.out
@@ -122,7 +122,7 @@ SELECT udf(count(*)) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(b)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(1))]; line 1 pos 75
+[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`udf(count(1))`]; line 1 pos 75
 
 
 -- !query
@@ -327,7 +327,7 @@ SELECT udf(count(udf(a))) FROM test_missing_target GROUP BY udf(a) ORDER BY udf(
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'b' does not exist. Did you mean one of the following? [udf(count(udf(a)))]; line 1 pos 80
+[UNRESOLVED_COLUMN] A column or function parameter with name `b` cannot be resolved. Did you mean one of the following? [`udf(count(udf(a)))`]; line 1 pos 80
 
 
 -- !query
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 753556c911b..4ffefd1b6a8 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
@@ -199,7 +199,7 @@ SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'k' does not exist. Did you mean one of the following? [testdata.a, testdata.b]; line 1 pos 57
+[UNRESOLVED_COLUMN] A column or function parameter with name `k` cannot be resolved. Did you mean one of the following? [`testdata`.`a`, `testdata`.`b`]; line 1 pos 57
 
 
 -- !query
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 7f92e489e22..875e14a25df 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
@@ -229,7 +229,7 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 'year' does not exist. Did you mean one of the following? [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0
+[UNRESOLVED_COLUMN] A column or function parameter with name `year` cannot be resolved. Did you mean one of the following? [`__auto_generated_subquery_name`.`course`, `__auto_generated_subquery_name`.`earnings`]; line 4 pos 0
 
 
 -- !query
@@ -323,7 +323,7 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-[MISSING_COLUMN] Column 's' does not exist. Did you mean one of the following? [coursesales.year, coursesales.course, coursesales.earnings]; line 4 pos 15
+[UNRESOLVED_COLUMN] A column or function parameter with name `s` cannot be resolved. Did you mean one of the following? [`coursesales`.`year`, `coursesales`.`course`, `coursesales`.`earnings`]; line 4 pos 15
 
 
 -- !query
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 4d82d110a4c..9924fbfbf62 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -2437,8 +2437,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex3 = intercept[AnalysisException] {
       df.selectExpr("transform(a, x -> x)")
     }
-    assert(ex3.getErrorClass == "MISSING_COLUMN")
-    assert(ex3.messageParameters.head == "a")
+    assert(ex3.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex3.messageParameters.head == "`a`")
   }
 
   test("map_filter") {
@@ -2509,8 +2509,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex4 = intercept[AnalysisException] {
       df.selectExpr("map_filter(a, (k, v) -> k > v)")
     }
-    assert(ex4.getErrorClass == "MISSING_COLUMN")
-    assert(ex4.messageParameters.head == "a")
+    assert(ex4.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4.messageParameters.head == "`a`")
   }
 
   test("filter function - array for primitive type not containing null") {
@@ -2669,8 +2669,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex4 = intercept[AnalysisException] {
       df.selectExpr("filter(a, x -> x)")
     }
-    assert(ex4.getErrorClass == "MISSING_COLUMN")
-    assert(ex4.messageParameters.head == "a")
+    assert(ex4.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4.messageParameters.head == "`a`")
   }
 
   test("exists function - array for primitive type not containing null") {
@@ -2802,8 +2802,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex4 = intercept[AnalysisException] {
       df.selectExpr("exists(a, x -> x)")
     }
-    assert(ex4.getErrorClass == "MISSING_COLUMN")
-    assert(ex4.messageParameters.head == "a")
+    assert(ex4.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4.messageParameters.head == "`a`")
   }
 
   test("forall function - array for primitive type not containing null") {
@@ -2949,14 +2949,14 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex4 = intercept[AnalysisException] {
       df.selectExpr("forall(a, x -> x)")
     }
-    assert(ex4.getErrorClass == "MISSING_COLUMN")
-    assert(ex4.messageParameters.head == "a")
+    assert(ex4.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4.messageParameters.head == "`a`")
 
     val ex4a = intercept[AnalysisException] {
       df.select(forall(col("a"), x => x))
     }
-    assert(ex4a.getErrorClass == "MISSING_COLUMN")
-    assert(ex4a.messageParameters.head == "a")
+    assert(ex4a.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4a.messageParameters.head == "`a`")
   }
 
   test("aggregate function - array for primitive type not containing null") {
@@ -3133,8 +3133,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex5 = intercept[AnalysisException] {
       df.selectExpr("aggregate(a, 0, (acc, x) -> x)")
     }
-    assert(ex5.getErrorClass == "MISSING_COLUMN")
-    assert(ex5.messageParameters.head == "a")
+    assert(ex5.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex5.messageParameters.head == "`a`")
   }
 
   test("map_zip_with function - map of primitive types") {
@@ -3687,8 +3687,8 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
     val ex4 = intercept[AnalysisException] {
       df.selectExpr("zip_with(a1, a, (acc, x) -> x)")
     }
-    assert(ex4.getErrorClass == "MISSING_COLUMN")
-    assert(ex4.messageParameters.head == "a")
+    assert(ex4.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(ex4.messageParameters.head == "`a`")
   }
 
   private def assertValuesDoNotChangeAfterCoalesceOrUnion(v: Column): Unit = {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
index 4d0dd46b956..5052dc0fc7f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSelfJoinSuite.scala
@@ -481,7 +481,10 @@ class DataFrameSelfJoinSuite extends QueryTest with SharedSparkSession {
       val ex = intercept[AnalysisException](
         df3.join(df1, year($"df1.timeStr") === year($"df3.tsStr"))
       )
-      assert(ex.message.contains("Column 'df1.timeStr' does not exist."))
+      checkError(ex,
+        errorClass = "UNRESOLVED_COLUMN",
+        parameters = Map("objectName" -> "`df1`.`timeStr`",
+          "objectList" -> "`df3`.`timeStr`, `df1`.`tsStr`"))
     }
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
index d7dc945ae13..4daa0a1b3b6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala
@@ -2673,8 +2673,8 @@ class DataFrameSuite extends QueryTest
     val err = intercept[AnalysisException] {
       df.groupBy($"d", $"b").as[GroupByKey, Row]
     }
-    assert(err.getErrorClass == "MISSING_COLUMN")
-    assert(err.messageParameters.head == "d")
+    assert(err.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(err.messageParameters.head == "`d`")
   }
 
   test("emptyDataFrame should be foldable") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
index 25d676f5d93..53a510fb142 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala
@@ -404,8 +404,8 @@ class DataFrameWindowFunctionsSuite extends QueryTest
     val df = Seq((1, "1")).toDF("key", "value")
     val e = intercept[AnalysisException](
       df.select($"key", count("invalid").over()))
-    assert(e.getErrorClass == "MISSING_COLUMN")
-    assert(e.messageParameters.sameElements(Array("invalid", "value, key")))
+    assert(e.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(e.messageParameters.sameElements(Array("`invalid`", "`value`, `key`")))
   }
 
   test("numerical aggregate functions on string column") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index a9cc66f8c37..463e79166fc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -325,13 +325,13 @@ class DatasetSuite extends QueryTest
       var e = intercept[AnalysisException] {
         ds.select(expr("`(_1)?+.+`").as[Int])
       }
-      assert(e.getErrorClass == "MISSING_COLUMN")
+      assert(e.getErrorClass == "UNRESOLVED_COLUMN")
       assert(e.messageParameters.head == "`(_1)?+.+`")
 
       e = intercept[AnalysisException] {
         ds.select(expr("`(_1|_2)`").as[Int])
       }
-      assert(e.getErrorClass == "MISSING_COLUMN")
+      assert(e.getErrorClass == "UNRESOLVED_COLUMN")
       assert(e.messageParameters.head == "`(_1|_2)`")
 
       e = intercept[AnalysisException] {
@@ -934,8 +934,8 @@ class DatasetSuite extends QueryTest
     val e = intercept[AnalysisException] {
       ds.as[ClassData2]
     }
-    assert(e.getErrorClass == "MISSING_COLUMN")
-    assert(e.messageParameters.sameElements(Array("c", "a, b")))
+    assert(e.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(e.messageParameters.sameElements(Array("`c`", "`a`, `b`")))
   }
 
   test("runtime nullability check") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
index 11abca1ddc7..7fd6a5dbea0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala
@@ -167,8 +167,8 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils {
       createTable("t1", cols, Seq("int", "long", "string"))
       val e1 = intercept[AnalysisException](sql(s"INSERT INTO t1 (c1, c2, c4) values(1, 2, 3)"))
       assert(e1.getMessage.contains(
-        "[MISSING_COLUMN] Column 'c4' does not exist. Did you mean one of the following? " +
-          "[c1, c2, c3]"))
+        "[UNRESOLVED_COLUMN] A column or function parameter with name `c4` cannot be resolved. " +
+          "Did you mean one of the following? [`c1`, `c2`, `c3`]"))
     }
   }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
index d2357c3d6c0..b1057fd14bc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala
@@ -887,8 +887,8 @@ class SubquerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
     withTempView("t") {
       Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("t")
       val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)"))
-      assert(e.getErrorClass == "MISSING_COLUMN")
-      assert(e.messageParameters.sameElements(Array("a", "t.i, t.j")))
+      assert(e.getErrorClass == "UNRESOLVED_COLUMN")
+      assert(e.messageParameters.sameElements(Array("`a`", "`t`.`i`, `t`.`j`")))
     }
   }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
index 8fd7aa3e0cd..ba8af623893 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala
@@ -725,8 +725,8 @@ class UDFSuite extends QueryTest with SharedSparkSession {
       .select(lit(50).as("a"))
       .select(struct("a").as("col"))
     val error = intercept[AnalysisException](df.select(myUdf(Column("col"))))
-    assert(error.getErrorClass == "MISSING_COLUMN")
-    assert(error.messageParameters.sameElements(Array("b", "a")))
+    assert(error.getErrorClass == "UNRESOLVED_COLUMN")
+    assert(error.messageParameters.sameElements(Array("`b`", "`a`")))
   }
 
   test("wrong order of input fields for case class") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 45c721100f7..00bb37dcfb2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -175,8 +175,8 @@ class DataSourceV2SQLSuite
 
       assertAnalysisErrorClass(
         s"DESCRIBE $t invalid_col",
-        "MISSING_COLUMN",
-        Array("invalid_col", "testcat.tbl.id, testcat.tbl.data"))
+        "UNRESOLVED_COLUMN",
+        Array("`invalid_col`", "`testcat`.`tbl`.`id`, `testcat`.`tbl`.`data`"))
     }
   }
 
@@ -1050,8 +1050,8 @@ class DataSourceV2SQLSuite
       val ex = intercept[AnalysisException] {
         sql(s"SELECT ns1.ns2.ns3.tbl.id from $t")
       }
-      assert(ex.getErrorClass == "MISSING_COLUMN")
-      assert(ex.messageParameters.head == "ns1.ns2.ns3.tbl.id")
+      assert(ex.getErrorClass == "UNRESOLVED_COLUMN")
+      assert(ex.messageParameters.head == "`ns1`.`ns2`.`ns3`.`tbl`.`id`")
     }
   }
 
@@ -1710,18 +1710,18 @@ class DataSourceV2SQLSuite
       // UPDATE non-existing column
       assertAnalysisErrorClass(
         s"UPDATE $t SET dummy='abc'",
-        "MISSING_COLUMN",
+        "UNRESOLVED_COLUMN",
         Array(
-          "dummy",
-          "testcat.ns1.ns2.tbl.p, testcat.ns1.ns2.tbl.id, " +
-            "testcat.ns1.ns2.tbl.age, testcat.ns1.ns2.tbl.name"))
+          "`dummy`",
+          "`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " +
+            "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`"))
       assertAnalysisErrorClass(
         s"UPDATE $t SET name='abc' WHERE dummy=1",
-        "MISSING_COLUMN",
+        "UNRESOLVED_COLUMN",
         Array(
-          "dummy",
-          "testcat.ns1.ns2.tbl.p, testcat.ns1.ns2.tbl.id, " +
-            "testcat.ns1.ns2.tbl.age, testcat.ns1.ns2.tbl.name"))
+          "`dummy`",
+          "`testcat`.`ns1`.`ns2`.`tbl`.`p`, `testcat`.`ns1`.`ns2`.`tbl`.`id`, " +
+            "`testcat`.`ns1`.`ns2`.`tbl`.`age`, `testcat`.`ns1`.`ns2`.`tbl`.`name`"))
 
       // UPDATE is not implemented yet.
       val e = intercept[UnsupportedOperationException] {
@@ -2773,7 +2773,7 @@ class DataSourceV2SQLSuite
         assert(e2.getMessage.contains(errMsg))
       }
       checkSubqueryError("SELECT 1 FROM non_exist", "Table or view not found: non_exist")
-      checkSubqueryError("SELECT col", "MISSING_COLUMN")
+      checkSubqueryError("SELECT col", "UNRESOLVED_COLUMN")
       checkSubqueryError("SELECT 1, 2", "Scalar subquery must return only one column")
       checkSubqueryError("SELECT * FROM VALUES (1), (2)", "MULTI_VALUE_SUBQUERY_ERROR")
     }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
index 4f631292436..06e6bec3fd1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
@@ -401,7 +401,7 @@ class QueryCompilationErrorsSuite
     )
   }
 
-  test("MISSING_COLUMN: SELECT distinct does not work correctly " +
+  test("UNRESOLVED_COLUMN: SELECT distinct does not work correctly " +
     "if order by missing attribute") {
     checkAnswer(
       sql(
@@ -424,9 +424,9 @@ class QueryCompilationErrorsSuite
             |order by struct.a, struct.b
             |""".stripMargin)
       },
-      errorClass = "MISSING_COLUMN",
-      msg = """Column 'struct.a' does not exist. """ +
-        """Did you mean one of the following\? \[a, b\]; line 6 pos 9;
+      errorClass = "UNRESOLVED_COLUMN",
+      msg = """A column or function parameter with name `struct`.`a` cannot be resolved. """ +
+        """Did you mean one of the following\? \[`a`, `b`\]; line 6 pos 9;
            |'Sort \['struct.a ASC NULLS FIRST, 'struct.b ASC NULLS FIRST\], true
            |\+\- Distinct
            |   \+\- Project \[struct#\w+\.a AS a#\w+, struct#\w+\.b AS b#\w+\]
@@ -440,16 +440,17 @@ class QueryCompilationErrorsSuite
       matchMsg = true)
   }
 
-  test("MISSING_COLUMN - SPARK-21335: support un-aliased subquery") {
+  test("UNRESOLVED_COLUMN - SPARK-21335: support un-aliased subquery") {
     withTempView("v") {
       Seq(1 -> "a").toDF("i", "j").createOrReplaceTempView("v")
       checkAnswer(sql("SELECT i from (SELECT i FROM v)"), Row(1))
 
       checkErrorClass(
         exception = intercept[AnalysisException](sql("SELECT v.i from (SELECT i FROM v)")),
-        errorClass = "MISSING_COLUMN",
-        msg = """Column 'v.i' does not exist. Did you mean one of the following\? """ +
-          """\[__auto_generated_subquery_name.i\]; line 1 pos 7;
+        errorClass = "UNRESOLVED_COLUMN",
+        msg = "A column or function parameter with name `v`.`i` cannot be resolved. " +
+          """Did you mean one of the following\? """ +
+          """\[`__auto_generated_subquery_name`.`i`\]; line 1 pos 7;
             |'Project \['v.i\]
             |\+\- SubqueryAlias __auto_generated_subquery_name
             |   \+\- Project \[i#\w+\]
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
index c1abe8e90be..823c7e01195 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
@@ -871,8 +871,11 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
             val e = intercept[AnalysisException] {
               sql("SELECT * FROM v1")
             }
-            assert(e.getErrorClass == "MISSING_COLUMN")
-            assert(e.messageParameters.sameElements(Array("C1", "spark_catalog.default.t.c1")))
+            checkError(e,
+              errorClass = "UNRESOLVED_COLUMN",
+              parameters = Map(
+                "objectName" -> "`C1`",
+                "objectList" -> "`spark_catalog`.`default`.`t`.`c1`"))
           }
           withSQLConf(ORDER_BY_ORDINAL.key -> "false") {
             checkAnswer(sql("SELECT * FROM v2"), Seq(Row(3), Row(2), Row(1)))
@@ -890,8 +893,11 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
             val e = intercept[AnalysisException] {
               sql("SELECT * FROM v4")
             }
-            assert(e.getErrorClass == "MISSING_COLUMN")
-            assert(e.messageParameters.sameElements(Array("a", "spark_catalog.default.t.c1")))
+            checkError(e,
+              errorClass = "UNRESOLVED_COLUMN",
+              parameters = Map(
+                "objectName" -> "`a`",
+                "objectList" -> "`spark_catalog`.`default`.`t`.`c1`"))
           }
           withSQLConf(ANSI_ENABLED.key -> "true") {
             val e = intercept[ArithmeticException] {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
index 17e23b9f7b1..62dccaad1dd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala
@@ -2619,8 +2619,8 @@ abstract class CSVSuite
             val ex = intercept[AnalysisException] {
               readback.filter($"AAA" === 2 && $"bbb" === 3).collect()
             }
-            assert(ex.getErrorClass == "MISSING_COLUMN")
-            assert(ex.messageParameters.head == "AAA")
+            assert(ex.getErrorClass == "UNRESOLVED_COLUMN")
+            assert(ex.messageParameters.head == "`AAA`")
           }
         }
       }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
index bc7c6e56ece..3fe9c58c957 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala
@@ -3044,8 +3044,8 @@ abstract class JsonSuite
             val ex = intercept[AnalysisException] {
               readback.filter($"AAA" === 0 && $"bbb" === 1).collect()
             }
-            assert(ex.getErrorClass == "MISSING_COLUMN")
-            assert(ex.messageParameters.head == "AAA")
+            assert(ex.getErrorClass == "UNRESOLVED_COLUMN")
+            assert(ex.messageParameters.head == "`AAA`")
             // Schema inferring
             val readback2 = spark.read.json(path.getCanonicalPath)
             checkAnswer(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
index a1d00361dfc..7370cf4f28b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala
@@ -1024,7 +1024,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
   test("SPARK-38336 INSERT INTO statements with tables with default columns: negative tests") {
     object Errors {
       val COMMON_SUBSTRING = " has a DEFAULT value"
-      val COLUMN_DEFAULT_NOT_FOUND = "Column 'default' does not exist"
+      val COLUMN_DEFAULT_NOT_FOUND = "`default` cannot be resolved."
       val BAD_SUBQUERY = "cannot evaluate expression scalarsubquery() in inline table definition"
     }
     // The default value fails to analyze.
@@ -1278,7 +1278,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
         assert(intercept[AnalysisException] {
           sql("insert into t (I) select true from (select 1)")
         }.getMessage.contains(
-          "[MISSING_COLUMN] Column 'I' does not exist. Did you mean one of the following? [i, s]"))
+          "[UNRESOLVED_COLUMN] A column or function parameter with name `I` cannot be resolved. " +
+            "Did you mean one of the following? [`i`, `s`]"))
       }
     }
   }
@@ -1757,8 +1758,8 @@ class InsertSuite extends DataSourceTest with SharedSparkSession {
             |)
           """.stripMargin)
       }
-      assert(ex.getErrorClass == "MISSING_COLUMN")
-      assert(ex.messageParameters.head == "c3")
+      assert(ex.getErrorClass == "UNRESOLVED_COLUMN")
+      assert(ex.messageParameters.head == "`c3`")
     }
   }
 
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
index 76a66cfdeb7..e5ecc2c889c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala
@@ -123,8 +123,8 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton
              |)
           """.stripMargin)
       }
-      assert(ex.getErrorClass == "MISSING_COLUMN")
-      assert(ex.messageParameters.head == "c3")
+      assert(ex.getErrorClass == "UNRESOLVED_COLUMN")
+      assert(ex.messageParameters.head == "`c3`")
     }
   }
 


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