You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2022/09/26 07:26:51 UTC

[spark] branch master updated: [SPARK-40540][SQL] Migrate compilation errors onto error classes

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 39f60578dc3 [SPARK-40540][SQL] Migrate compilation errors onto error classes
39f60578dc3 is described below

commit 39f60578dc3c189e7f5cfb2a74f0502408aa123e
Author: Max Gekk <ma...@gmail.com>
AuthorDate: Mon Sep 26 10:26:27 2022 +0300

    [SPARK-40540][SQL] Migrate compilation errors onto error classes
    
    ### What changes were proposed in this pull request?
    In the PR, I propose to migrate 100 compilation errors onto temporary error classes with the prefix `_LEGACY_ERROR_TEMP_10xx`. The error message will not include the error classes, so, in this way we will preserve the existing behaviour.
    
    ### Why are the changes needed?
    The migration on temporary error classes allows to gather statistics about errors and detect most popular error classes. After that we could prioritise the work on migration.
    
    The new error class name prefix `_LEGACY_ERROR_TEMP_` proposed here kind of marks the error as developer-facing, not user-facing. Developers can still get the error class programmatically via the `SparkThrowable` interface, so that they can build error infra with it. End users won't see the error class in the message. This allows us to do the error migration very quickly, and we can refine the error classes and mark them as user-facing later (naming them properly, adding tests, etc.).
    
    ### Does this PR introduce _any_ user-facing change?
    No. The error messages should be almost the same by default.
    
    ### How was this patch tested?
    By running the modified test suites:
    ```
    $ build/sbt "core/testOnly *SparkThrowableSuite"
    $ build/sbt "sql/testOnly org.apache.spark.sql.SQLQueryTestSuite"
    $ build/sbt "test:testOnly *AnalysisErrorSuite"
    $ build/sbt "test:testOnly *.HiveUDAFSuite"
    $ build/sbt "test:testOnly *.DataFrameSuite"
    $ build/sbt "test:testOnly *SQLQuerySuite"
    ```
    
    Closes #37973 from MaxGekk/legacy-error-temp-compliation.
    
    Authored-by: Max Gekk <ma...@gmail.com>
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 core/src/main/resources/error/error-classes.json   | 502 +++++++++++++++++++
 .../spark/sql/errors/QueryCompilationErrors.scala  | 536 ++++++++++++++-------
 .../sql/catalyst/analysis/AnalysisErrorSuite.scala |   2 +-
 .../results/ansi/string-functions.sql.out          |  32 +-
 .../results/ceil-floor-with-scale-param.sql.out    |  32 +-
 .../sql-tests/results/change-column.sql.out        |  34 +-
 .../results/columnresolution-negative.sql.out      |  15 +-
 .../test/resources/sql-tests/results/count.sql.out |   7 +-
 .../sql-tests/results/csv-functions.sql.out        |  24 +-
 .../sql-tests/results/group-by-ordinal.sql.out     |   4 +-
 .../sql-tests/results/join-lateral.sql.out         |  15 +-
 .../sql-tests/results/json-functions.sql.out       |  72 ++-
 .../sql-tests/results/order-by-ordinal.sql.out     |  45 +-
 .../test/resources/sql-tests/results/pivot.sql.out |  14 +-
 .../results/postgreSQL/window_part3.sql.out        |  21 +-
 .../sql-tests/results/show_columns.sql.out         |   8 +-
 .../results/sql-compatibility-functions.sql.out    |  14 +-
 .../sql-tests/results/string-functions.sql.out     |  32 +-
 .../sql-tests/results/table-aliases.sql.out        |  30 +-
 .../results/table-valued-functions.sql.out         |   2 +-
 .../sql-tests/results/timestamp-ntz.sql.out        |  16 +-
 .../sql-tests/results/udaf/udaf-group-by.sql.out   |  15 +-
 .../resources/sql-tests/results/udaf/udaf.sql.out  |  31 +-
 .../sql-tests/results/udf/udf-pivot.sql.out        |  14 +-
 .../sql-tests/results/udf/udf-udaf.sql.out         |  31 +-
 .../sql-tests/results/udf/udf-window.sql.out       |   7 +-
 .../resources/sql-tests/results/window.sql.out     |  25 +-
 .../org/apache/spark/sql/DataFrameSuite.scala      |  20 +-
 .../scala/org/apache/spark/sql/SQLQuerySuite.scala |  41 +-
 .../spark/sql/connector/DataSourceV2SQLSuite.scala |  13 +-
 .../execution/command/DescribeTableSuiteBase.scala |   2 +-
 .../execution/command/v1/DescribeTableSuite.scala  |   4 +-
 .../spark/sql/hive/execution/HiveUDAFSuite.scala   |  18 +-
 33 files changed, 1428 insertions(+), 250 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index 5694b2c9d0f..aff6576af73 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -1163,5 +1163,507 @@
     "message" : [
       "<msg>."
     ]
+  },
+  "_LEGACY_ERROR_TEMP_1000" : {
+    "message" : [
+      "LEGACY store assignment policy is disallowed in Spark data source V2. Please set the configuration <configKey> to other values."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1001" : {
+    "message" : [
+      "USING column `<colName>` cannot be resolved on the <side> side of the join. The <side>-side columns: [<plan>]."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1002" : {
+    "message" : [
+      "Unable to generate an encoder for inner class `<className>` without access to the scope that this class was defined in.",
+      "Try moving this class out of its parent class."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1003" : {
+    "message" : [
+      "Couldn't find the reference column for <after> at <parentName>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1004" : {
+    "message" : [
+      "Window specification <windowName> is not defined in the WINDOW clause."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1005" : {
+    "message" : [
+      "<expr> doesn't show up in the GROUP BY list <groupByAliases>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1006" : {
+    "message" : [
+      "Aggregate expression required for pivot, but '<sql>' did not appear in any aggregate function."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1007" : {
+    "message" : [
+      "Cannot write into temp view <quoted> as it's not a data source v2 relation."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1008" : {
+    "message" : [
+      "<quoted> is not a temp view of streaming logical plan, please use batch API such as `DataFrameReader.table` to read it."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1009" : {
+    "message" : [
+      "The depth of view <identifier> exceeds the maximum view resolution depth (<maxNestedViewDepth>). Analysis is aborted to avoid errors. Increase the value of <config> to work around this."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1010" : {
+    "message" : [
+      "Inserting into a view is not allowed. View: <identifier>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1011" : {
+    "message" : [
+      "Writing into a view is not allowed. View: <identifier>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1012" : {
+    "message" : [
+      "Cannot write into v1 table: <identifier>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1013" : {
+    "message" : [
+      "<nameParts> is a <viewStr>. '<cmd>' expects a table.<hintStr>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1014" : {
+    "message" : [
+      "<nameParts> is a temp view. '<cmd>' expects a permanent view."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1015" : {
+    "message" : [
+      "<identifier> is a table. '<cmd>' expects a view.<hintStr>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1016" : {
+    "message" : [
+      "<nameParts> is a temp view. '<cmd>' expects a table or permanent view."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1017" : {
+    "message" : [
+      "<name> is a built-in/temporary function. '<cmd>' expects a persistent function.<hintStr>"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1018" : {
+    "message" : [
+      "<quoted> is a permanent view, which is not supported by streaming reading API such as `DataStreamReader.table` yet."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1019" : {
+    "message" : [
+      "Star (*) is not allowed in select list when GROUP BY ordinal position is used"
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1020" : {
+    "message" : [
+      "Invalid usage of <elem> in <prettyName>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1021" : {
+    "message" : [
+      "count(<targetString>.*) is not allowed. Please use count(*) or expand the columns manually, e.g. count(col1, col2)."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1022" : {
+    "message" : [
+      "ORDER BY position <index> is not in select list (valid range is [1, <size>])."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1023" : {
+    "message" : [
+      "Function <prettyName> does not support <syntax>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1024" : {
+    "message" : [
+      "FILTER expression is non-deterministic, it cannot be used in aggregate functions."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1025" : {
+    "message" : [
+      "FILTER expression is not of type boolean. It cannot be used in an aggregate function."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1026" : {
+    "message" : [
+      "FILTER expression contains aggregate. It cannot be used in an aggregate function."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1027" : {
+    "message" : [
+      "FILTER expression contains window function. It cannot be used in an aggregate function."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1028" : {
+    "message" : [
+      "Number of column aliases does not match number of columns. Number of column aliases: <columnSize>; number of columns: <outputSize>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1029" : {
+    "message" : [
+      "The number of aliases supplied in the AS clause does not match the number of columns output by the UDTF expected <aliasesSize> aliases but got <aliasesNames>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1030" : {
+    "message" : [
+      "Window aggregate function with filter predicate is not supported yet."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1031" : {
+    "message" : [
+      "It is not allowed to use a window function inside an aggregate function. Please use the inner window function in a sub-query."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1032" : {
+    "message" : [
+      "<expr> does not have any WindowExpression."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1033" : {
+    "message" : [
+      "<expr> has multiple Window Specifications (<distinctWindowSpec>).",
+      "Please file a bug report with this error message, stack trace, and the query."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1034" : {
+    "message" : [
+      "It is not allowed to use window functions inside <clauseName> clause."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1035" : {
+    "message" : [
+      "Cannot specify window frame for <prettyName> function."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1036" : {
+    "message" : [
+      "Window Frame <wf> must match the required frame <required>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1037" : {
+    "message" : [
+      "Window function <wf> requires window to be ordered, please add ORDER BY clause. For example SELECT <wf>(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1038" : {
+    "message" : [
+      "Cannot write to table due to mismatched user specified column size(<columnSize>) and data column size(<outputSize>)."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1039" : {
+    "message" : [
+      "Multiple time/session window expressions would result in a cartesian product of rows, therefore they are currently not supported."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1040" : {
+    "message" : [
+      "Gap duration expression used in session window must be CalendarIntervalType, but got <dt>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1041" : {
+    "message" : [
+      "Undefined function <name>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1042" : {
+    "message" : [
+      "Invalid number of arguments for function <name>. Expected: <expectedInfo>; Found: <actualNumber>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1043" : {
+    "message" : [
+      "Invalid arguments for function <name>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1044" : {
+    "message" : [
+      "Function <name> accepts only one argument."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1045" : {
+    "message" : [
+      "ALTER TABLE SET LOCATION does not support partition for v2 tables."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1046" : {
+    "message" : [
+      "Join strategy hint parameter should be an identifier or string but was <unsupported> (<class>)."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1047" : {
+    "message" : [
+      "<hintName> Hint parameter should include columns, but <invalidParams> found."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1048" : {
+    "message" : [
+      "<hintName> Hint expects a partition number as a parameter."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1049" : {
+    "message" : [
+      "Syntax error in attribute name: <name>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1050" : {
+    "message" : [
+      "Can only star expand struct data types. Attribute: `<attributes>`."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1051" : {
+    "message" : [
+      "Cannot resolve '<targetString>.*' given input columns '<columns>'."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1052" : {
+    "message" : [
+      "ADD COLUMN with v1 tables cannot specify NOT NULL."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1053" : {
+    "message" : [
+      "ALTER COLUMN with v1 tables cannot specify NOT NULL."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1054" : {
+    "message" : [
+      "ALTER COLUMN cannot find column <colName> in v1 table. Available: <fieldNames>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1055" : {
+    "message" : [
+      "The database name is not valid: <quoted>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1056" : {
+    "message" : [
+      "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1057" : {
+    "message" : [
+      "SHOW COLUMNS with conflicting databases: '<dbA>' != '<dbB>'."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1058" : {
+    "message" : [
+      "Cannot create table with both USING <provider> and <serDeInfo>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1059" : {
+    "message" : [
+      "STORED AS with file format '<serdeInfo>' is invalid."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1060" : {
+    "message" : [
+      "<command> does not support nested column: <column>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1061" : {
+    "message" : [
+      "Column <colName> does not exist."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1062" : {
+    "message" : [
+      "Rename temporary view from '<oldName>' to '<newName>': destination view already exists."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1063" : {
+    "message" : [
+      "Cannot drop a non-empty database: <db>. Use CASCADE option to drop a non-empty database."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1064" : {
+    "message" : [
+      "Cannot drop a non-empty namespace: <namespace>. Use CASCADE option to drop a non-empty namespace."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1065" : {
+    "message" : [
+      "`<name>` is not a valid name for tables/databases. Valid names only contain alphabet characters, numbers and _."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1066" : {
+    "message" : [
+      "<database> is a system preserved database, you cannot create a database with this name."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1067" : {
+    "message" : [
+      "Can not drop default database."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1068" : {
+    "message" : [
+      "<database> is a system preserved database, you cannot use it as current database. To access global temporary views, you should use qualified name with the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM <database>.viewName."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1069" : {
+    "message" : [
+      "CREATE EXTERNAL TABLE must be accompanied by LOCATION."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1070" : {
+    "message" : [
+      "Can not <methodName> the managed table('<tableIdentifier>'). The associated location('<tableLocation>') already exists."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1071" : {
+    "message" : [
+      "Some existing schema fields (<nonExistentColumnNames>) are not present in the new schema. We don't support dropping columns yet."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1072" : {
+    "message" : [
+      "Only the tables/views belong to the same database can be retrieved. Querying tables/views are <qualifiedTableNames>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1073" : {
+    "message" : [
+      "RENAME TABLE source and destination databases do not match: '<db>' != '<newDb>'."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1074" : {
+    "message" : [
+      "RENAME TEMPORARY VIEW from '<oldName>' to '<newName>': cannot specify database name '<db>' in the destination table."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1075" : {
+    "message" : [
+      "RENAME TEMPORARY VIEW from '<oldName>' to '<newName>': destination table already exists."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1076" : {
+    "message" : [
+      "Partition spec is invalid. <details>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1077" : {
+    "message" : [
+      "Function <func> already exists."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1078" : {
+    "message" : [
+      "Can not load class '<className>' when registering the function '<func>', please make sure it is on the classpath."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1079" : {
+    "message" : [
+      "Resource Type '<resourceType>' is not supported."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1080" : {
+    "message" : [
+      "Table <identifier> did not specify database."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1081" : {
+    "message" : [
+      "Table <identifier> did not specify locationUri."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1082" : {
+    "message" : [
+      "Partition [<specString>] did not specify locationUri."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1083" : {
+    "message" : [
+      "Number of buckets should be greater than 0 but less than or equal to bucketing.maxBuckets (`<bucketingMaxBuckets>`). Got `<numBuckets>`."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1084" : {
+    "message" : [
+      "Corrupted table name context in catalog: <numParts> parts expected, but part <index> is missing."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1085" : {
+    "message" : [
+      "Corrupted view SQL configs in catalog."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1086" : {
+    "message" : [
+      "Corrupted view query output column names in catalog: <numCols> parts expected, but part <index> is missing."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1087" : {
+    "message" : [
+      "Corrupted view referred temp view names in catalog."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1088" : {
+    "message" : [
+      "Corrupted view referred temp functions names in catalog."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1089" : {
+    "message" : [
+      "Column statistics deserialization is not supported for column <name> of data type: <dataType>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1090" : {
+    "message" : [
+      "Column statistics serialization is not supported for column <colName> of data type: <dataType>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1091" : {
+    "message" : [
+      "Cannot read table property '<key>' as it's corrupted.<details>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1092" : {
+    "message" : [
+      "The expression '<expr>' is not a valid schema string."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1093" : {
+    "message" : [
+      "Schema should be specified in DDL format as a string literal or output of the schema_of_json/schema_of_csv functions instead of <expr>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1094" : {
+    "message" : [
+      "Schema should be struct type but got <dataType>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1095" : {
+    "message" : [
+      "A type of keys and values in map() must be string, but got <map>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1096" : {
+    "message" : [
+      "Must use a map() function for options."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1097" : {
+    "message" : [
+      "The field for corrupt records must be string type and nullable."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1098" : {
+    "message" : [
+      "DataType '<x>' is not supported by <className>."
+    ]
+  },
+  "_LEGACY_ERROR_TEMP_1099" : {
+    "message" : [
+      "<funcName>() doesn't support the <mode> mode. Acceptable modes are <permissiveMode> and <failFastMode>."
+    ]
   }
 }
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 af39d8860e5..cf7ce045bff 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
@@ -164,15 +164,18 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   def legacyStoreAssignmentPolicyError(): Throwable = {
     val configKey = SQLConf.STORE_ASSIGNMENT_POLICY.key
     new AnalysisException(
-      "LEGACY store assignment policy is disallowed in Spark data source V2. " +
-        s"Please set the configuration $configKey to other values.")
+      errorClass = "_LEGACY_ERROR_TEMP_1000",
+      messageParameters = Map("configKey" -> configKey))
   }
 
   def unresolvedUsingColForJoinError(
       colName: String, plan: LogicalPlan, side: String): Throwable = {
     new AnalysisException(
-      s"USING column `$colName` cannot be resolved on the $side " +
-        s"side of the join. The $side-side columns: [${plan.output.map(_.name).mkString(", ")}]")
+      errorClass = "_LEGACY_ERROR_TEMP_1001",
+      messageParameters = Map(
+        "colName" -> colName,
+        "side" -> side,
+        "plan" -> plan.output.map(_.name).mkString(", ")))
   }
 
   def unresolvedAttributeError(
@@ -262,23 +265,29 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
 
   def outerScopeFailureForNewInstanceError(className: String): Throwable = {
     new AnalysisException(
-      s"Unable to generate an encoder for inner class `$className` without " +
-        "access to the scope that this class was defined in.\n" +
-        "Try moving this class out of its parent class.")
+      errorClass = "_LEGACY_ERROR_TEMP_1002",
+      messageParameters = Map("className" -> className))
   }
 
   def referenceColNotFoundForAlterTableChangesError(
       after: TableChange.After, parentName: String): Throwable = {
     new AnalysisException(
-      s"Couldn't find the reference column for $after at $parentName")
+      errorClass = "_LEGACY_ERROR_TEMP_1003",
+      messageParameters = Map("after" -> after.toString, "parentName" -> parentName))
   }
 
   def windowSpecificationNotDefinedError(windowName: String): Throwable = {
-    new AnalysisException(s"Window specification $windowName is not defined in the WINDOW clause.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1004",
+      messageParameters = Map("windowName" -> windowName))
   }
 
   def selectExprNotInGroupByError(expr: Expression, groupByAliases: Seq[Alias]): Throwable = {
-    new AnalysisException(s"$expr doesn't show up in the GROUP BY list $groupByAliases")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1005",
+      messageParameters = Map(
+        "expr" -> expr.toString,
+        "groupByAliases" -> groupByAliases.toString()))
   }
 
   def groupingMustWithGroupingSetsOrCubeOrRollupError(): Throwable = {
@@ -295,41 +304,53 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def aggregateExpressionRequiredForPivotError(sql: String): Throwable = {
-    new AnalysisException(s"Aggregate expression required for pivot, but '$sql' " +
-      "did not appear in any aggregate function.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1006",
+      messageParameters = Map("sql" -> sql))
   }
 
   def writeIntoTempViewNotAllowedError(quoted: String): Throwable = {
-    new AnalysisException("Cannot write into temp view " +
-      s"$quoted as it's not a data source v2 relation.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1007",
+      messageParameters = Map("quoted" -> quoted))
   }
 
   def readNonStreamingTempViewError(quoted: String): Throwable = {
-    new AnalysisException(s"$quoted is not a temp view of streaming " +
-      "logical plan, please use batch API such as `DataFrameReader.table` to read it.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1008",
+      messageParameters = Map("quoted" -> quoted))
   }
 
   def viewDepthExceedsMaxResolutionDepthError(
       identifier: TableIdentifier, maxNestedViewDepth: Int, t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"The depth of view $identifier exceeds the maximum " +
-      s"view resolution depth ($maxNestedViewDepth). Analysis is aborted to " +
-      s"avoid errors. Increase the value of ${SQLConf.MAX_NESTED_VIEW_DEPTH.key} to work " +
-      "around this.", t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1009",
+      messageParameters = Map(
+        "identifier" -> identifier.toString,
+        "maxNestedViewDepth" -> maxNestedViewDepth.toString,
+        "config" -> SQLConf.MAX_NESTED_VIEW_DEPTH.key),
+      origin = t.origin)
   }
 
   def insertIntoViewNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"Inserting into a view is not allowed. View: $identifier.",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1010",
+      messageParameters = Map("identifier" -> identifier.toString),
+      origin = t.origin)
   }
 
   def writeIntoViewNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"Writing into a view is not allowed. View: $identifier.",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1011",
+      messageParameters = Map("identifier" -> identifier.toString),
+      origin = t.origin)
   }
 
   def writeIntoV1TableNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"Cannot write into v1 table: $identifier.",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1012",
+      messageParameters = Map("identifier" -> identifier.toString),
+      origin = t.origin)
   }
 
   def expectTableNotViewError(
@@ -340,48 +361,72 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       t: TreeNode[_]): Throwable = {
     val viewStr = if (isTemp) "temp view" else "view"
     val hintStr = mismatchHint.map(" " + _).getOrElse("")
-    new AnalysisException(s"${nameParts.quoted} is a $viewStr. '$cmd' expects a table.$hintStr",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1013",
+      messageParameters = Map(
+        "nameParts" -> nameParts.quoted,
+        "viewStr" -> viewStr,
+        "cmd" -> cmd,
+        "hintStr" -> hintStr),
+      origin = t.origin)
   }
 
   def expectViewNotTempViewError(
       nameParts: Seq[String],
       cmd: String,
       t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"${nameParts.quoted} is a temp view. '$cmd' expects a permanent view.",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1014",
+      messageParameters = Map(
+        "nameParts" -> nameParts.quoted,
+        "cmd" -> cmd),
+      origin = t.origin)
   }
 
   def expectViewNotTableError(
       v: ResolvedTable, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = {
     val hintStr = mismatchHint.map(" " + _).getOrElse("")
-    new AnalysisException(s"${v.identifier.quoted} is a table. '$cmd' expects a view.$hintStr",
-      t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1015",
+      messageParameters = Map(
+        "identifier" -> v.identifier.quoted,
+        "cmd" -> cmd,
+        "hintStr" -> hintStr),
+      origin = t.origin)
   }
 
   def expectTableOrPermanentViewNotTempViewError(
       nameParts: Seq[String], cmd: String, t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      s"${nameParts.quoted} is a temp view. '$cmd' expects a table or permanent view.",
-      t.origin.line, t.origin.startPosition)
+      errorClass = "_LEGACY_ERROR_TEMP_1016",
+      messageParameters = Map(
+        "nameParts" -> nameParts.quoted,
+        "cmd" -> cmd),
+      origin = t.origin)
   }
 
   def expectPersistentFuncError(
       name: String, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = {
     val hintStr = mismatchHint.map(" " + _).getOrElse("")
     new AnalysisException(
-      s"$name is a built-in/temporary function. '$cmd' expects a persistent function.$hintStr",
-      t.origin.line, t.origin.startPosition)
+      errorClass = "_LEGACY_ERROR_TEMP_1017",
+      messageParameters = Map(
+        "name" -> name,
+        "cmd" -> cmd,
+        "hintStr" -> hintStr),
+      origin = t.origin)
   }
 
   def permanentViewNotSupportedByStreamingReadingAPIError(quoted: String): Throwable = {
-    new AnalysisException(s"$quoted is a permanent view, which is not supported by " +
-      "streaming reading API such as `DataStreamReader.table` yet.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1018",
+      messageParameters = Map("quoted" -> quoted))
   }
 
   def starNotAllowedWhenGroupByOrdinalPositionUsedError(): Throwable = {
     new AnalysisException(
-      "Star (*) is not allowed in select list when GROUP BY ordinal position is used")
+      errorClass = "_LEGACY_ERROR_TEMP_1019",
+      messageParameters = Map.empty)
   }
 
   def invalidStarUsageError(prettyName: String, stars: Seq[Star]): Throwable = {
@@ -396,17 +441,24 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
       None
     }
     val elem = Seq(starMsg, resExprMsg).flatten.mkString(" and ")
-    new AnalysisException(s"Invalid usage of $elem in $prettyName")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1020",
+      messageParameters = Map("elem" -> elem, "prettyName" -> prettyName))
   }
 
   def singleTableStarInCountNotAllowedError(targetString: String): Throwable = {
-    new AnalysisException(s"count($targetString.*) is not allowed. " +
-      "Please use count(*) or expand the columns manually, e.g. count(col1, col2)")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1021",
+      messageParameters = Map("targetString" -> targetString))
   }
 
   def orderByPositionRangeError(index: Int, size: Int, t: TreeNode[_]): Throwable = {
-    new AnalysisException(s"ORDER BY position $index is not in select list " +
-      s"(valid range is [1, $size])", t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1022",
+      messageParameters = Map(
+        "index" -> index.toString,
+        "size" -> size.toString),
+      origin = t.origin)
   }
 
   def groupByPositionRefersToAggregateFunctionError(
@@ -436,112 +488,153 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def functionWithUnsupportedSyntaxError(prettyName: String, syntax: String): Throwable = {
-    new AnalysisException(s"Function $prettyName does not support $syntax")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1023",
+      messageParameters = Map("prettyName" -> prettyName, "syntax" -> syntax))
   }
 
   def nonDeterministicFilterInAggregateError(): Throwable = {
-    new AnalysisException("FILTER expression is non-deterministic, " +
-      "it cannot be used in aggregate functions")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1024",
+      messageParameters = Map.empty)
   }
 
   def nonBooleanFilterInAggregateError(): Throwable = {
-    new AnalysisException("FILTER expression is not of type boolean. " +
-      "It cannot be used in an aggregate function")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1025",
+      messageParameters = Map.empty)
   }
 
   def aggregateInAggregateFilterError(): Throwable = {
-    new AnalysisException("FILTER expression contains aggregate. " +
-      "It cannot be used in an aggregate function")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1026",
+      messageParameters = Map.empty)
   }
 
   def windowFunctionInAggregateFilterError(): Throwable = {
-    new AnalysisException("FILTER expression contains window function. " +
-      "It cannot be used in an aggregate function")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1027",
+      messageParameters = Map.empty)
   }
 
   def aliasNumberNotMatchColumnNumberError(
       columnSize: Int, outputSize: Int, t: TreeNode[_]): Throwable = {
-    new AnalysisException("Number of column aliases does not match number of columns. " +
-      s"Number of column aliases: $columnSize; " +
-      s"number of columns: $outputSize.", t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1028",
+      messageParameters = Map(
+        "columnSize" -> columnSize.toString,
+        "outputSize" -> outputSize.toString),
+      origin = t.origin)
   }
 
   def aliasesNumberNotMatchUDTFOutputError(
       aliasesSize: Int, aliasesNames: String): Throwable = {
-    new AnalysisException("The number of aliases supplied in the AS clause does not " +
-      s"match the number of columns output by the UDTF expected $aliasesSize " +
-      s"aliases but got $aliasesNames ")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1029",
+      messageParameters = Map(
+        "aliasesSize" -> aliasesSize.toString,
+        "aliasesNames" -> aliasesNames))
   }
 
   def windowAggregateFunctionWithFilterNotSupportedError(): Throwable = {
-    new AnalysisException("window aggregate function with filter predicate is not supported yet.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1030",
+      messageParameters = Map.empty)
   }
 
   def windowFunctionInsideAggregateFunctionNotAllowedError(): Throwable = {
-    new AnalysisException("It is not allowed to use a window function inside an aggregate " +
-      "function. Please use the inner window function in a sub-query.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1031",
+      messageParameters = Map.empty)
   }
 
   def expressionWithoutWindowExpressionError(expr: NamedExpression): Throwable = {
-    new AnalysisException(s"$expr does not have any WindowExpression.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1032",
+      messageParameters = Map("expr" -> expr.toString))
   }
 
   def expressionWithMultiWindowExpressionsError(
       expr: NamedExpression, distinctWindowSpec: Seq[WindowSpecDefinition]): Throwable = {
-    new AnalysisException(s"$expr has multiple Window Specifications ($distinctWindowSpec)." +
-      "Please file a bug report with this error message, stack trace, and the query.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1033",
+      messageParameters = Map(
+        "expr" -> expr.toString,
+        "distinctWindowSpec" -> distinctWindowSpec.toString()))
   }
 
   def windowFunctionNotAllowedError(clauseName: String): Throwable = {
-    new AnalysisException(s"It is not allowed to use window functions inside $clauseName clause")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1034",
+      messageParameters = Map("clauseName" -> clauseName))
   }
 
   def cannotSpecifyWindowFrameError(prettyName: String): Throwable = {
-    new AnalysisException(s"Cannot specify window frame for $prettyName function")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1035",
+      messageParameters = Map("prettyName" -> prettyName))
   }
 
   def windowFrameNotMatchRequiredFrameError(
       f: SpecifiedWindowFrame, required: WindowFrame): Throwable = {
-    new AnalysisException(s"Window Frame $f must match the required frame $required")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1036",
+      messageParameters = Map(
+        "wf" -> f.toString,
+        "required" -> required.toString))
   }
 
   def windowFunctionWithWindowFrameNotOrderedError(wf: WindowFunction): Throwable = {
-    new AnalysisException(s"Window function $wf requires window to be ordered, please add " +
-      s"ORDER BY clause. For example SELECT $wf(value_expr) OVER (PARTITION BY window_partition " +
-      "ORDER BY window_ordering) from table")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1037",
+      messageParameters = Map("wf" -> wf.toString))
   }
 
   def writeTableWithMismatchedColumnsError(
       columnSize: Int, outputSize: Int, t: TreeNode[_]): Throwable = {
-    new AnalysisException("Cannot write to table due to mismatched user specified column " +
-      s"size($columnSize) and data column size($outputSize)", t.origin.line, t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1038",
+      messageParameters = Map(
+        "columnSize" -> columnSize.toString,
+        "outputSize" -> outputSize.toString),
+      origin = t.origin)
   }
 
   def multiTimeWindowExpressionsNotSupportedError(t: TreeNode[_]): Throwable = {
-    new AnalysisException("Multiple time/session window expressions would result in a cartesian " +
-      "product of rows, therefore they are currently not supported.", t.origin.line,
-      t.origin.startPosition)
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1039",
+      messageParameters = Map.empty,
+      origin = t.origin)
   }
 
   def sessionWindowGapDurationDataTypeError(dt: DataType): Throwable = {
-    new AnalysisException("Gap duration expression used in session window must be " +
-      s"CalendarIntervalType, but got ${dt}")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1040",
+      messageParameters = Map("dt" -> dt.toString))
   }
 
   def functionUndefinedError(name: FunctionIdentifier): Throwable = {
-    new AnalysisException(s"undefined function $name")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1041",
+      messageParameters = Map("name" -> name.toString))
   }
 
   def invalidFunctionArgumentsError(
       name: String, expectedInfo: String, actualNumber: Int): Throwable = {
-    new AnalysisException(s"Invalid number of arguments for function $name. " +
-      s"Expected: $expectedInfo; Found: $actualNumber")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1042",
+      messageParameters = Map(
+        "name" -> name,
+        "expectedInfo" -> expectedInfo,
+        "actualNumber" -> actualNumber.toString))
   }
 
   def invalidFunctionArgumentNumberError(
       validParametersCount: Seq[Int], name: String, actualNumber: Int): Throwable = {
     if (validParametersCount.length == 0) {
-      new AnalysisException(s"Invalid arguments for function $name")
+      new AnalysisException(
+        errorClass = "_LEGACY_ERROR_TEMP_1043",
+        messageParameters = Map("name" -> name))
     } else {
       val expectedNumberOfParameters = if (validParametersCount.length == 1) {
         validParametersCount.head.toString
@@ -554,43 +647,65 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def functionAcceptsOnlyOneArgumentError(name: String): Throwable = {
-    new AnalysisException(s"Function $name accepts only one argument")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1044",
+      messageParameters = Map("name" -> name))
   }
 
   def alterV2TableSetLocationWithPartitionNotSupportedError(): Throwable = {
-    new AnalysisException("ALTER TABLE SET LOCATION does not support partition for v2 tables.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1045",
+      messageParameters = Map.empty)
   }
 
   def joinStrategyHintParameterNotSupportedError(unsupported: Any): Throwable = {
-    new AnalysisException("Join strategy hint parameter " +
-      s"should be an identifier or string but was $unsupported (${unsupported.getClass}")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1046",
+      messageParameters = Map(
+        "unsupported" -> unsupported.toString,
+        "class" -> unsupported.getClass.toString))
   }
 
   def invalidHintParameterError(
       hintName: String, invalidParams: Seq[Any]): Throwable = {
-    new AnalysisException(s"$hintName Hint parameter should include columns, but " +
-      s"${invalidParams.mkString(", ")} found")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1047",
+      messageParameters = Map(
+        "hintName" -> hintName,
+        "invalidParams" -> invalidParams.mkString(", ")))
   }
 
   def invalidCoalesceHintParameterError(hintName: String): Throwable = {
-    new AnalysisException(s"$hintName Hint expects a partition number as a parameter")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1048",
+      messageParameters = Map("hintName" -> hintName))
   }
 
   def attributeNameSyntaxError(name: String): Throwable = {
-    new AnalysisException(s"syntax error in attribute name: $name")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1049",
+      messageParameters = Map("name" -> name))
   }
 
   def starExpandDataTypeNotSupportedError(attributes: Seq[String]): Throwable = {
-    new AnalysisException(s"Can only star expand struct data types. Attribute: `$attributes`")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1050",
+      messageParameters = Map("attributes" -> attributes.toString()))
   }
 
   def cannotResolveStarExpandGivenInputColumnsError(
       targetString: String, columns: String): Throwable = {
-    new AnalysisException(s"cannot resolve '$targetString.*' given input columns '$columns'")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1051",
+      messageParameters = Map(
+        "targetString" -> targetString,
+        "columns" -> columns))
   }
 
   def addColumnWithV1TableCannotSpecifyNotNullError(): Throwable = {
-    new AnalysisException("ADD COLUMN with v1 tables cannot specify NOT NULL.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1052",
+      messageParameters = Map.empty)
   }
 
   def operationOnlySupportedWithV2TableError(
@@ -614,234 +729,327 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase {
   }
 
   def alterColumnWithV1TableCannotSpecifyNotNullError(): Throwable = {
-    new AnalysisException("ALTER COLUMN with v1 tables cannot specify NOT NULL.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1053",
+      messageParameters = Map.empty)
   }
 
   def alterColumnCannotFindColumnInV1TableError(colName: String, v1Table: V1Table): Throwable = {
     new AnalysisException(
-      s"ALTER COLUMN cannot find column $colName in v1 table. " +
-        s"Available: ${v1Table.schema.fieldNames.mkString(", ")}")
+      errorClass = "_LEGACY_ERROR_TEMP_1054",
+      messageParameters = Map(
+        "colName" -> colName,
+        "fieldNames" -> v1Table.schema.fieldNames.mkString(", ")))
   }
 
   def invalidDatabaseNameError(quoted: String): Throwable = {
-    new AnalysisException(s"The database name is not valid: $quoted")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1055",
+      messageParameters = Map("database" -> quoted))
   }
 
   def cannotDropViewWithDropTableError(): Throwable = {
-    new AnalysisException("Cannot drop a view with DROP TABLE. Please use DROP VIEW instead")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1056",
+      messageParameters = Map.empty)
   }
 
   def showColumnsWithConflictDatabasesError(
       db: Seq[String], v1TableName: TableIdentifier): Throwable = {
-    new AnalysisException("SHOW COLUMNS with conflicting databases: " +
-        s"'${db.head}' != '${v1TableName.database.get}'")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1057",
+      messageParameters = Map(
+        "dbA" -> db.head,
+        "dbB" -> v1TableName.database.get))
   }
 
   def cannotCreateTableWithBothProviderAndSerdeError(
       provider: Option[String], maybeSerdeInfo: Option[SerdeInfo]): Throwable = {
     new AnalysisException(
-      s"Cannot create table with both USING $provider and ${maybeSerdeInfo.get.describe}")
+      errorClass = "_LEGACY_ERROR_TEMP_1058",
+      messageParameters = Map(
+        "provider" -> provider.toString,
+        "serDeInfo" -> maybeSerdeInfo.get.describe))
   }
 
   def invalidFileFormatForStoredAsError(serdeInfo: SerdeInfo): Throwable = {
     new AnalysisException(
-      s"STORED AS with file format '${serdeInfo.storedAs.get}' is invalid.")
+      errorClass = "_LEGACY_ERROR_TEMP_1059",
+      messageParameters = Map("serdeInfo" -> serdeInfo.storedAs.get))
   }
 
   def commandNotSupportNestedColumnError(command: String, quoted: String): Throwable = {
-    new AnalysisException(s"$command does not support nested column: $quoted")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1060",
+      messageParameters = Map(
+        "command" -> command,
+        "column" -> quoted))
   }
 
   def columnDoesNotExistError(colName: String): Throwable = {
-    new AnalysisException(s"Column $colName does not exist")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1061",
+      messageParameters = Map("colName" -> colName))
   }
 
   def renameTempViewToExistingViewError(oldName: String, newName: String): Throwable = {
     new AnalysisException(
-      s"rename temporary view from '$oldName' to '$newName': destination view already exists")
+      errorClass = "_LEGACY_ERROR_TEMP_1062",
+      messageParameters = Map("oldName" -> oldName, "newName" -> newName))
   }
 
   def cannotDropNonemptyDatabaseError(db: String): Throwable = {
-    new AnalysisException(s"Cannot drop a non-empty database: $db. " +
-      "Use CASCADE option to drop a non-empty database.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1063",
+      messageParameters = Map("db" -> db))
   }
 
   def cannotDropNonemptyNamespaceError(namespace: Seq[String]): Throwable = {
-    new AnalysisException(s"Cannot drop a non-empty namespace: ${namespace.quoted}. " +
-      "Use CASCADE option to drop a non-empty namespace.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1064",
+      messageParameters = Map("namespace" -> namespace.quoted))
   }
 
   def invalidNameForTableOrDatabaseError(name: String): Throwable = {
-    new AnalysisException(s"`$name` is not a valid name for tables/databases. " +
-      "Valid names only contain alphabet characters, numbers and _.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1065",
+      messageParameters = Map("name" -> name))
   }
 
   def cannotCreateDatabaseWithSameNameAsPreservedDatabaseError(database: String): Throwable = {
-    new AnalysisException(s"$database is a system preserved database, " +
-      "you cannot create a database with this name.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1066",
+      messageParameters = Map("database" -> database))
   }
 
   def cannotDropDefaultDatabaseError(): Throwable = {
-    new AnalysisException("Can not drop default database")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1067",
+      messageParameters = Map.empty)
   }
 
   def cannotUsePreservedDatabaseAsCurrentDatabaseError(database: String): Throwable = {
-    new AnalysisException(s"$database is a system preserved database, you cannot use it as " +
-      "current database. To access global temporary views, you should use qualified name with " +
-      s"the GLOBAL_TEMP_DATABASE, e.g. SELECT * FROM $database.viewName.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1068",
+      messageParameters = Map("database" -> database))
   }
 
   def createExternalTableWithoutLocationError(): Throwable = {
-    new AnalysisException("CREATE EXTERNAL TABLE must be accompanied by LOCATION")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1069",
+      messageParameters = Map.empty)
   }
 
   def cannotOperateManagedTableWithExistingLocationError(
       methodName: String, tableIdentifier: TableIdentifier, tableLocation: Path): Throwable = {
-    new AnalysisException(s"Can not $methodName the managed table('$tableIdentifier')" +
-      s". The associated location('${tableLocation.toString}') already exists.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1070",
+      messageParameters = Map(
+        "methodName" -> methodName,
+        "tableIdentifier" -> tableIdentifier.toString,
+        "tableLocation" -> tableLocation.toString))
   }
 
   def dropNonExistentColumnsNotSupportedError(
       nonExistentColumnNames: Seq[String]): Throwable = {
     new AnalysisException(
-      s"""
-         |Some existing schema fields (${nonExistentColumnNames.mkString("[", ",", "]")}) are
-         |not present in the new schema. We don't support dropping columns yet.
-         """.stripMargin)
+      errorClass = "_LEGACY_ERROR_TEMP_1071",
+      messageParameters = Map(
+        "nonExistentColumnNames" -> nonExistentColumnNames.mkString("[", ",", "]")))
   }
 
   def cannotRetrieveTableOrViewNotInSameDatabaseError(
       qualifiedTableNames: Seq[QualifiedTableName]): Throwable = {
-    new AnalysisException("Only the tables/views belong to the same database can be retrieved. " +
-      s"Querying tables/views are $qualifiedTableNames")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1072",
+      messageParameters = Map("qualifiedTableNames" -> qualifiedTableNames.toString()))
   }
 
   def renameTableSourceAndDestinationMismatchError(db: String, newDb: String): Throwable = {
     new AnalysisException(
-      s"RENAME TABLE source and destination databases do not match: '$db' != '$newDb'")
+      errorClass = "_LEGACY_ERROR_TEMP_1073",
+      messageParameters = Map("db" -> db, "newDb" -> newDb))
   }
 
   def cannotRenameTempViewWithDatabaseSpecifiedError(
       oldName: TableIdentifier, newName: TableIdentifier): Throwable = {
-    new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': cannot " +
-      s"specify database name '${newName.database.get}' in the destination table")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1074",
+      messageParameters = Map(
+        "oldName" -> oldName.toString,
+        "newName" -> newName.toString,
+        "db" -> newName.database.get))
   }
 
   def cannotRenameTempViewToExistingTableError(
       oldName: TableIdentifier, newName: TableIdentifier): Throwable = {
-    new AnalysisException(s"RENAME TEMPORARY VIEW from '$oldName' to '$newName': " +
-      "destination table already exists")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1075",
+      messageParameters = Map(
+        "oldName" -> oldName.toString,
+        "newName" -> newName.toString))
   }
 
   def invalidPartitionSpecError(details: String): Throwable = {
-    new AnalysisException(s"Partition spec is invalid. $details")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1076",
+      messageParameters = Map("details" -> details))
   }
 
   def functionAlreadyExistsError(func: FunctionIdentifier): Throwable = {
-    new AnalysisException(s"Function $func already exists")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1077",
+      messageParameters = Map("func" -> func.toString))
   }
 
   def cannotLoadClassWhenRegisteringFunctionError(
       className: String, func: FunctionIdentifier): Throwable = {
-    new AnalysisException(s"Can not load class '$className' when registering " +
-      s"the function '$func', please make sure it is on the classpath")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1078",
+      messageParameters = Map(
+        "className" -> className,
+        "func" -> func.toString))
   }
 
   def resourceTypeNotSupportedError(resourceType: String): Throwable = {
-    new AnalysisException(s"Resource Type '$resourceType' is not supported.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1079",
+      messageParameters = Map("resourceType" -> resourceType))
   }
 
   def tableNotSpecifyDatabaseError(identifier: TableIdentifier): Throwable = {
-    new AnalysisException(s"table $identifier did not specify database")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1080",
+      messageParameters = Map("identifier" -> identifier.toString))
   }
 
   def tableNotSpecifyLocationUriError(identifier: TableIdentifier): Throwable = {
-    new AnalysisException(s"table $identifier did not specify locationUri")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1081",
+      messageParameters = Map("identifier" -> identifier.toString))
   }
 
   def partitionNotSpecifyLocationUriError(specString: String): Throwable = {
-    new AnalysisException(s"Partition [$specString] did not specify locationUri")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1082",
+      messageParameters = Map("specString" -> specString))
   }
 
   def invalidBucketNumberError(bucketingMaxBuckets: Int, numBuckets: Int): Throwable = {
     new AnalysisException(
-      s"Number of buckets should be greater than 0 but less than or equal to " +
-        s"bucketing.maxBuckets (`$bucketingMaxBuckets`). Got `$numBuckets`")
+      errorClass = "_LEGACY_ERROR_TEMP_1083",
+      messageParameters = Map(
+        "bucketingMaxBuckets" -> bucketingMaxBuckets.toString,
+        "numBuckets" -> numBuckets.toString))
   }
 
   def corruptedTableNameContextInCatalogError(numParts: Int, index: Int): Throwable = {
-    new AnalysisException("Corrupted table name context in catalog: " +
-      s"$numParts parts expected, but part $index is missing.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1084",
+      messageParameters = Map(
+        "numParts" -> numParts.toString,
+        "index" -> index.toString))
   }
 
   def corruptedViewSQLConfigsInCatalogError(e: Exception): Throwable = {
-    new AnalysisException("Corrupted view SQL configs in catalog", cause = Some(e))
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1085",
+      messageParameters = Map.empty,
+      cause = Some(e))
   }
 
   def corruptedViewQueryOutputColumnsInCatalogError(numCols: String, index: Int): Throwable = {
-    new AnalysisException("Corrupted view query output column names in catalog: " +
-      s"$numCols parts expected, but part $index is missing.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1086",
+      messageParameters = Map(
+        "numCols" -> numCols,
+        "index" -> index.toString))
   }
 
   def corruptedViewReferredTempViewInCatalogError(e: Exception): Throwable = {
-    new AnalysisException("corrupted view referred temp view names in catalog", cause = Some(e))
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1087",
+      messageParameters = Map.empty,
+      cause = Some(e))
   }
 
   def corruptedViewReferredTempFunctionsInCatalogError(e: Exception): Throwable = {
-    new AnalysisException(
-      "corrupted view referred temp functions names in catalog", cause = Some(e))
+        new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1088",
+      messageParameters = Map.empty,
+      cause = Some(e))
   }
 
   def columnStatisticsDeserializationNotSupportedError(
       name: String, dataType: DataType): Throwable = {
-    new AnalysisException("Column statistics deserialization is not supported for " +
-      s"column $name of data type: $dataType.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1089",
+      messageParameters = Map("name" -> name, "dataType" -> dataType.toString))
   }
 
   def columnStatisticsSerializationNotSupportedError(
       colName: String, dataType: DataType): Throwable = {
-    new AnalysisException("Column statistics serialization is not supported for " +
-      s"column $colName of data type: $dataType.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1090",
+      messageParameters = Map("colName" -> colName, "dataType" -> dataType.toString))
   }
 
   def cannotReadCorruptedTablePropertyError(key: String, details: String = ""): Throwable = {
-    new AnalysisException(s"Cannot read table property '$key' as it's corrupted.$details")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1091",
+      messageParameters = Map("key" -> key, "details" -> details))
   }
 
   def invalidSchemaStringError(exp: Expression): Throwable = {
-    new AnalysisException(s"The expression '${exp.sql}' is not a valid schema string.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1092",
+      messageParameters = Map("expr" -> exp.sql))
   }
 
   def schemaNotFoldableError(exp: Expression): Throwable = {
     new AnalysisException(
-      "Schema should be specified in DDL format as a string literal or output of " +
-        s"the schema_of_json/schema_of_csv functions instead of ${exp.sql}")
+      errorClass = "_LEGACY_ERROR_TEMP_1093",
+      messageParameters = Map("expr" -> exp.sql))
   }
 
   def schemaIsNotStructTypeError(dataType: DataType): Throwable = {
-    new AnalysisException(s"Schema should be struct type but got ${dataType.sql}.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1094",
+      messageParameters = Map("dataType" -> dataType.toString))
   }
 
   def keyValueInMapNotStringError(m: CreateMap): Throwable = {
     new AnalysisException(
-      s"A type of keys and values in map() must be string, but got ${m.dataType.catalogString}")
+      errorClass = "_LEGACY_ERROR_TEMP_1095",
+      messageParameters = Map("map" -> m.dataType.catalogString))
   }
 
   def nonMapFunctionNotAllowedError(): Throwable = {
-    new AnalysisException("Must use a map() function for options")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1096",
+      messageParameters = Map.empty)
   }
 
   def invalidFieldTypeForCorruptRecordError(): Throwable = {
-    new AnalysisException("The field for corrupt records must be string type and nullable")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1097",
+      messageParameters = Map.empty)
   }
 
   def dataTypeUnsupportedByClassError(x: DataType, className: String): Throwable = {
-    new AnalysisException(s"DataType '$x' is not supported by $className.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1098",
+      messageParameters = Map("x" -> x.toString, "className" -> className))
   }
 
   def parseModeUnsupportedError(funcName: String, mode: ParseMode): Throwable = {
-    new AnalysisException(s"$funcName() doesn't support the ${mode.name} mode. " +
-      s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}.")
+    new AnalysisException(
+      errorClass = "_LEGACY_ERROR_TEMP_1099",
+      messageParameters = Map(
+        "funcName" -> funcName,
+        "mode" -> mode.name,
+        "permissiveMode" -> PermissiveMode.name,
+        "failFastMode" -> FailFastMode.name))
   }
 
   def requireLiteralParameter(
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 2a8c830d8d7..9fdb061b7ff 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
@@ -340,7 +340,7 @@ class AnalysisErrorSuite extends AnalysisTest {
   errorTest(
     "unresolved star expansion in max",
     testRelation2.groupBy($"a")(sum(UnresolvedStar(None))),
-    "Invalid usage of '*'" :: "in expression 'sum'" :: Nil)
+    "Invalid usage of '*' in expression 'sum'." :: Nil)
 
   errorTest(
     "sorting by unsupported column types",
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out
index 4e04a29a98c..2379d5e35e7 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out
@@ -758,7 +758,21 @@ select decode()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "2",
+    "name" : "decode"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 15,
+    "fragment" : "decode()"
+  } ]
+}
 
 
 -- !query
@@ -767,7 +781,21 @@ select decode(encode('abc', 'utf-8'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "1",
+    "expectedInfo" : "2",
+    "name" : "decode"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 37,
+    "fragment" : "decode(encode('abc', 'utf-8'))"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out
index 38efdac4092..c1af10e5ec1 100644
--- a/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ceil-floor-with-scale-param.sql.out
@@ -111,7 +111,21 @@ SELECT CEIL(2.5, 0, 0)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function ceil. Expected: 2; Found: 3; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "3",
+    "expectedInfo" : "2",
+    "name" : "ceil"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 22,
+    "fragment" : "CEIL(2.5, 0, 0)"
+  } ]
+}
 
 
 -- !query
@@ -226,4 +240,18 @@ SELECT FLOOR(2.5, 0, 0)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function floor. Expected: 2; Found: 3; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "3",
+    "expectedInfo" : "2",
+    "name" : "floor"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 23,
+    "fragment" : "FLOOR(2.5, 0, 0)"
+  } ]
+}
diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
index 4f35bceb95b..9d2ef445e16 100644
--- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
@@ -254,7 +254,22 @@ ALTER TABLE temp_view CHANGE a TYPE INT
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-temp_view is a temp view. 'ALTER TABLE ... CHANGE COLUMN' expects a table.; line 1 pos 12
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "messageParameters" : {
+    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
+    "hintStr" : "",
+    "nameParts" : "temp_view",
+    "viewStr" : "temp view"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 13,
+    "stopIndex" : 21,
+    "fragment" : "temp_view"
+  } ]
+}
 
 
 -- !query
@@ -271,7 +286,22 @@ ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-global_temp.global_temp_view is a temp view. 'ALTER TABLE ... CHANGE COLUMN' expects a table.; line 1 pos 12
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "messageParameters" : {
+    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
+    "hintStr" : "",
+    "nameParts" : "global_temp.global_temp_view",
+    "viewStr" : "temp view"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 13,
+    "stopIndex" : 40,
+    "fragment" : "global_temp.global_temp_view"
+  } ]
+}
 
 
 -- !query
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 253637eecf4..86c1d0590e2 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
@@ -222,7 +222,20 @@ SELECT t1.x.y.* FROM t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 't1.x.y.*' given input columns 'i1'; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "i1",
+    "targetString" : "t1.x.y"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 15,
+    "fragment" : "t1.x.y.*"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/count.sql.out
index ab9b543a9f8..183c5561126 100644
--- a/sql/core/src/test/resources/sql-tests/results/count.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/count.sql.out
@@ -179,4 +179,9 @@ SELECT count(testData.*) FROM testData
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-count(testData.*) is not allowed. Please use count(*) or expand the columns manually, e.g. count(col1, col2)
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1021",
+  "messageParameters" : {
+    "targetString" : "testData"
+  }
+}
diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
index 291739f9c3b..922073aae86 100644
--- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
@@ -52,7 +52,7 @@ select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+Must use a map() function for options.; line 1 pos 7
 
 
 -- !query
@@ -61,7 +61,7 @@ select from_csv('1', 'a INT', map('mode', 1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+A type of keys and values in map() must be string, but got map<string,int>.; line 1 pos 7
 
 
 -- !query
@@ -70,7 +70,21 @@ select from_csv()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "one of 2 and 3",
+    "name" : "from_csv"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 17,
+    "fragment" : "from_csv()"
+  } ]
+}
 
 
 -- !query
@@ -173,7 +187,7 @@ select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+Must use a map() function for options.; line 1 pos 7
 
 
 -- !query
@@ -182,4 +196,4 @@ select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+A type of keys and values in map() must be string, but got map<string,int>.; line 1 pos 7
diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
index e21b79eb046..666c932c158 100644
--- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out
@@ -222,7 +222,9 @@ select * from data group by a, b, 1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Star (*) is not allowed in select list when GROUP BY ordinal position is used
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1019"
+}
 
 
 -- !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 a3d9a6a6c1d..14631bc6682 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
@@ -114,7 +114,20 @@ SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2, LATERAL (SELECT t1.*, t2.*
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-cannot resolve 't1.*' given input columns 'c1, c2'; line 1 pos 70
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1051",
+  "messageParameters" : {
+    "columns" : "c1, c2",
+    "targetString" : "t1"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 71,
+    "stopIndex" : 74,
+    "fragment" : "t1.*"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
index ec6d3871178..6f0a5beaeef 100644
--- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
@@ -69,7 +69,7 @@ select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+Must use a map() function for options.; line 1 pos 7
 
 
 -- !query
@@ -78,7 +78,7 @@ select to_json(named_struct('a', 1, 'b', 2), map('mode', 1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+A type of keys and values in map() must be string, but got map<string,int>.; line 1 pos 7
 
 
 -- !query
@@ -87,7 +87,21 @@ select to_json()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "one of 1 and 2",
+    "name" : "to_json"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 16,
+    "fragment" : "to_json()"
+  } ]
+}
 
 
 -- !query
@@ -143,7 +157,7 @@ select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Must use a map() function for options; line 1 pos 7
+Must use a map() function for options.; line 1 pos 7
 
 
 -- !query
@@ -152,7 +166,7 @@ select from_json('{"a":1}', 'a INT', map('mode', 1))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-A type of keys and values in map() must be string, but got map<string,int>; line 1 pos 7
+A type of keys and values in map() must be string, but got map<string,int>.; line 1 pos 7
 
 
 -- !query
@@ -161,7 +175,21 @@ select from_json()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "one of 2 and 3",
+    "name" : "from_json"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 18,
+    "fragment" : "from_json()"
+  } ]
+}
 
 
 -- !query
@@ -491,7 +519,21 @@ select json_array_length()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function json_array_length. Expected: 1; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "1",
+    "name" : "json_array_length"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 26,
+    "fragment" : "json_array_length()"
+  } ]
+}
 
 
 -- !query
@@ -564,7 +606,21 @@ select json_object_keys()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function json_object_keys. Expected: 1; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "1",
+    "name" : "json_object_keys"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 25,
+    "fragment" : "json_object_keys()"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out
index 581d7d4ae2d..685aacef279 100644
--- a/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/order-by-ordinal.sql.out
@@ -72,7 +72,20 @@ select * from data order by 0
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-ORDER BY position 0 is not in select list (valid range is [1, 2]); line 1 pos 28
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1022",
+  "messageParameters" : {
+    "index" : "0",
+    "size" : "2"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 29,
+    "stopIndex" : 29,
+    "fragment" : "0"
+  } ]
+}
 
 
 -- !query
@@ -81,7 +94,20 @@ select * from data order by -1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-ORDER BY position -1 is not in select list (valid range is [1, 2]); line 1 pos 28
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1022",
+  "messageParameters" : {
+    "index" : "-1",
+    "size" : "2"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 29,
+    "stopIndex" : 30,
+    "fragment" : "-1"
+  } ]
+}
 
 
 -- !query
@@ -90,7 +116,20 @@ select * from data order by 3
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-ORDER BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 28
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1022",
+  "messageParameters" : {
+    "index" : "3",
+    "size" : "2"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 29,
+    "stopIndex" : 29,
+    "fragment" : "3"
+  } ]
+}
 
 
 -- !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 b25362aba1e..efaaa4e31b9 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
@@ -199,7 +199,12 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Aggregate expression required for pivot, but 'coursesales.earnings' did not appear in any aggregate function.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1006",
+  "messageParameters" : {
+    "sql" : "coursesales.earnings"
+  }
+}
 
 
 -- !query
@@ -214,7 +219,12 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Aggregate expression required for pivot, but '__auto_generated_subquery_name.year' did not appear in any aggregate function.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1006",
+  "messageParameters" : {
+    "sql" : "__auto_generated_subquery_name.year"
+  }
+}
 
 
 -- !query
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 1d6e249ff86..226cd0a8c01 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
@@ -296,7 +296,12 @@ SELECT * FROM empsalary WHERE row_number() OVER (ORDER BY salary) < 10
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-It is not allowed to use window functions inside WHERE clause
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1034",
+  "messageParameters" : {
+    "clauseName" : "WHERE"
+  }
+}
 
 
 -- !query
@@ -345,7 +350,12 @@ SELECT * FROM empsalary WHERE (rank() OVER (ORDER BY random())) > 10
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-It is not allowed to use window functions inside WHERE clause
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1034",
+  "messageParameters" : {
+    "clauseName" : "WHERE"
+  }
+}
 
 
 -- !query
@@ -354,7 +364,12 @@ SELECT * FROM empsalary WHERE rank() OVER (ORDER BY random())
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-It is not allowed to use window functions inside WHERE clause
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1034",
+  "messageParameters" : {
+    "clauseName" : "WHERE"
+  }
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out
index 8a858343b24..7624d1800f5 100644
--- a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out
@@ -109,7 +109,13 @@ SHOW COLUMNS IN showdb.showcolumn1 FROM baddb
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-SHOW COLUMNS with conflicting databases: 'baddb' != 'showdb'
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1057",
+  "messageParameters" : {
+    "dbA" : "baddb",
+    "dbB" : "showdb"
+  }
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out
index 9521cbf0bf2..e0d5874d058 100644
--- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out
@@ -93,7 +93,19 @@ SELECT string(1, 2)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Function string accepts only one argument; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1044",
+  "messageParameters" : {
+    "name" : "string"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 19,
+    "fragment" : "string(1, 2)"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out
index 8502c60ce6e..70dc4362517 100644
--- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out
@@ -690,7 +690,21 @@ select decode()
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 0; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "0",
+    "expectedInfo" : "2",
+    "name" : "decode"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 15,
+    "fragment" : "decode()"
+  } ]
+}
 
 
 -- !query
@@ -699,7 +713,21 @@ select decode(encode('abc', 'utf-8'))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function decode. Expected: 2; Found: 1; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "1",
+    "expectedInfo" : "2",
+    "name" : "decode"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 37,
+    "fragment" : "decode(encode('abc', 'utf-8'))"
+  } ]
+}
 
 
 -- !query
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 b38d742b03f..a94bf53d141 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
@@ -39,7 +39,20 @@ SELECT * FROM testData AS t(col1, col2, col3)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Number of column aliases does not match number of columns. Number of column aliases: 3; number of columns: 2.; line 1 pos 14
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1028",
+  "messageParameters" : {
+    "columnSize" : "3",
+    "outputSize" : "2"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 15,
+    "stopIndex" : 45,
+    "fragment" : "testData AS t(col1, col2, col3)"
+  } ]
+}
 
 
 -- !query
@@ -48,7 +61,20 @@ SELECT * FROM testData AS t(col1)
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Number of column aliases does not match number of columns. Number of column aliases: 1; number of columns: 2.; line 1 pos 14
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1028",
+  "messageParameters" : {
+    "columnSize" : "1",
+    "outputSize" : "2"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 15,
+    "stopIndex" : 33,
+    "fragment" : "testData AS t(col1)"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
index 1ce6fbbdbc8..c9e632f2dc5 100644
--- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
@@ -72,7 +72,7 @@ Table-valued function range with alternatives:
     range(start: long, end: long, step: long)
     range(start: long, end: long)
     range(end: long)
-cannot be applied to (integer, integer, integer, integer, integer): Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5; line 1 pos 14
+cannot be applied to (integer, integer, integer, integer, integer): Invalid number of arguments for function range. Expected: one of 1, 2, 3 and 4; Found: 5.; line 1 pos 14
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out
index acdc65a23f4..dd0ea1ab687 100644
--- a/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/timestamp-ntz.sql.out
@@ -45,7 +45,21 @@ SELECT make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET')
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function make_timestamp_ntz. Expected: 6; Found: 7; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "7",
+    "expectedInfo" : "6",
+    "name" : "make_timestamp_ntz"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 61,
+    "fragment" : "make_timestamp_ntz(2021, 07, 11, 6, 30, 45.678, 'CET')"
+  } ]
+}
 
 
 -- !query
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 bb98d244faa..2ce1ca514ad 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
@@ -146,7 +146,20 @@ SELECT udaf(DISTINCT b), udaf(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Function pythonudf does not support DISTINCT; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1023",
+  "messageParameters" : {
+    "prettyName" : "pythonudf",
+    "syntax" : "DISTINCT"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 23,
+    "fragment" : "udaf(DISTINCT b)"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/udaf/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf/udaf.sql.out
index 0605af1c808..69221c731a9 100644
--- a/sql/core/src/test/resources/sql-tests/results/udaf/udaf.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udaf/udaf.sql.out
@@ -31,7 +31,21 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function spark_catalog.default.mydoubleavg. Expected: 1; Found: 2; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "2",
+    "expectedInfo" : "1",
+    "name" : "spark_catalog.default.mydoubleavg"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 39,
+    "fragment" : "default.myDoubleAvg(int_col1, 3)"
+  } ]
+}
 
 
 -- !query
@@ -48,7 +62,20 @@ SELECT default.udaf1(int_col1) as udaf1 from t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Can not load class 'test.non.existent.udaf' when registering the function 'spark_catalog.default.udaf1', please make sure it is on the classpath; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1078",
+  "messageParameters" : {
+    "className" : "test.non.existent.udaf",
+    "func" : "spark_catalog.default.udaf1"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 30,
+    "fragment" : "default.udaf1(int_col1)"
+  } ]
+}
 
 
 -- !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 3390b139c5b..6fa03ce672e 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
@@ -199,7 +199,12 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Aggregate expression required for pivot, but 'coursesales.earnings' did not appear in any aggregate function.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1006",
+  "messageParameters" : {
+    "sql" : "coursesales.earnings"
+  }
+}
 
 
 -- !query
@@ -214,7 +219,12 @@ PIVOT (
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Aggregate expression required for pivot, but '__auto_generated_subquery_name.year' did not appear in any aggregate function.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1006",
+  "messageParameters" : {
+    "sql" : "__auto_generated_subquery_name.year"
+  }
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out
index 80a3d9af942..899905a41cd 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out
@@ -31,7 +31,21 @@ SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Invalid number of arguments for function spark_catalog.default.mydoubleavg. Expected: 1; Found: 2; line 1 pos 7
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1042",
+  "messageParameters" : {
+    "actualNumber" : "2",
+    "expectedInfo" : "1",
+    "name" : "spark_catalog.default.mydoubleavg"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 8,
+    "stopIndex" : 49,
+    "fragment" : "default.myDoubleAvg(udf(int_col1), udf(3))"
+  } ]
+}
 
 
 -- !query
@@ -48,7 +62,20 @@ SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1)))
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Can not load class 'test.non.existent.udaf' when registering the function 'spark_catalog.default.udaf1', please make sure it is on the classpath; line 1 pos 94
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1078",
+  "messageParameters" : {
+    "className" : "test.non.existent.udaf",
+    "func" : "spark_catalog.default.udaf1"
+  },
+  "queryContext" : [ {
+    "objectType" : "",
+    "objectName" : "",
+    "startIndex" : 95,
+    "stopIndex" : 117,
+    "fragment" : "default.udaf1(int_col1)"
+  } ]
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out
index 94e10b3a067..a8e5b08714e 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out
@@ -327,7 +327,12 @@ SELECT udf(val), cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1037",
+  "messageParameters" : {
+    "wf" : "row_number()"
+  }
+}
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out
index 2b358d221e6..3719657dce3 100644
--- a/sql/core/src/test/resources/sql-tests/results/window.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out
@@ -490,7 +490,12 @@ SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY ca
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1037",
+  "messageParameters" : {
+    "wf" : "row_number()"
+  }
+}
 
 
 -- !query
@@ -562,7 +567,9 @@ FROM testData ORDER BY cate, val
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-window aggregate function with filter predicate is not supported yet.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1030"
+}
 
 
 -- !query
@@ -1210,7 +1217,12 @@ FROM
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Window specification w is not defined in the WINDOW clause.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1004",
+  "messageParameters" : {
+    "windowName" : "w"
+  }
+}
 
 
 -- !query
@@ -1222,4 +1234,9 @@ FROM
 struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
-Window specification w is not defined in the WINDOW clause.
+{
+  "errorClass" : "_LEGACY_ERROR_TEMP_1004",
+  "messageParameters" : {
+    "windowName" : "w"
+  }
+}
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 1dcf0b6e4a9..b29b5c2b341 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
@@ -1622,17 +1622,19 @@ class DataFrameSuite extends QueryTest
         Row(1)
       )
 
-      def checkError(testFun: => Unit): Unit = {
-        val e = intercept[org.apache.spark.sql.AnalysisException] {
-          testFun
-        }
-        assert(e.getMessage.contains("syntax error in attribute name:"))
+      def checkSyntaxError(name: String): Unit = {
+        checkError(
+          exception = intercept[org.apache.spark.sql.AnalysisException] {
+            df(name)
+          },
+          errorClass = "_LEGACY_ERROR_TEMP_1049",
+          parameters = Map("name" -> name))
       }
 
-      checkError(df("`abc.`c`"))
-      checkError(df("`abc`..d"))
-      checkError(df("`a`.b."))
-      checkError(df("`a.b`.c.`d"))
+      checkSyntaxError("`abc.`c`")
+      checkSyntaxError("`abc`..d")
+      checkSyntaxError("`a`.b.")
+      checkSyntaxError("`a.b`.c.`d")
     }
   }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 215b005319e..12f8274fb66 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -1789,8 +1789,13 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
           nestedStructData.select($"record.r1.*"))
 
         // Try resolving something not there.
-        assert(intercept[AnalysisException](sql("SELECT abc.* FROM nestedStructTable"))
-          .getMessage.contains("cannot resolve"))
+        checkError(
+          exception = intercept[AnalysisException]{
+            sql("SELECT abc.* FROM nestedStructTable")
+          },
+          errorClass = "_LEGACY_ERROR_TEMP_1051",
+          parameters = Map("targetString" -> "abc", "columns" -> "record"),
+          context = ExpectedContext(fragment = "abc.*", start = 7, stop = 11))
       }
 
       // Create paths with unusual characters
@@ -1819,8 +1824,13 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
       }
 
       // Try star expanding a scalar. This should fail.
-      assert(intercept[AnalysisException](sql("select a.* from testData2")).getMessage.contains(
-        "Can only star expand struct data types."))
+      checkError(
+        exception = intercept[AnalysisException]{
+          sql("select a.* from testData2")
+        },
+        errorClass = "_LEGACY_ERROR_TEMP_1050",
+        parameters = Map("attributes" -> "ArrayBuffer(a)"),
+        context = ExpectedContext(fragment = "a.*", start = 7, stop = 9))
     }
   }
 
@@ -1866,15 +1876,20 @@ class SQLQuerySuite extends QueryTest with SharedSparkSession with AdaptiveSpark
         dfNoCols,
         dfNoCols.select($"*"))
 
-      var e = intercept[AnalysisException] {
-        sql("SELECT a.* FROM temp_table_no_cols a")
-      }.getMessage
-      assert(e.contains("cannot resolve 'a.*' given input columns ''"))
-
-      e = intercept[AnalysisException] {
-        dfNoCols.select($"b.*")
-      }.getMessage
-      assert(e.contains("cannot resolve 'b.*' given input columns ''"))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql("SELECT a.* FROM temp_table_no_cols a")
+        },
+        errorClass = "_LEGACY_ERROR_TEMP_1051",
+        parameters = Map("targetString" -> "a", "columns" -> ""),
+        context = ExpectedContext(fragment = "a.*", start = 7, stop = 9))
+
+      checkError(
+        exception = intercept[AnalysisException] {
+          dfNoCols.select($"b.*")
+        },
+        errorClass = "_LEGACY_ERROR_TEMP_1051",
+        parameters = Map("targetString" -> "b", "columns" -> ""))
     }
   }
 
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 092f371b339..bdf7c1a872b 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
@@ -2030,10 +2030,15 @@ class DataSourceV2SQLSuiteV1Filter extends DataSourceV2SQLSuite with AlterTableT
       sql("USE testcat.ns1.ns2")
       check("tbl")
 
-      val ex = intercept[AnalysisException] {
-        sql(s"SELECT ns1.ns2.ns3.tbl.* from $t")
-      }
-      assert(ex.getMessage.contains("cannot resolve 'ns1.ns2.ns3.tbl.*"))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"SELECT ns1.ns2.ns3.tbl.* from $t")
+        },
+        errorClass = "_LEGACY_ERROR_TEMP_1051",
+        parameters = Map(
+          "targetString" -> "ns1.ns2.ns3.tbl",
+          "columns" -> "id, name"),
+        context = ExpectedContext(fragment = "ns1.ns2.ns3.tbl.*", start = 7, stop = 23))
     }
   }
 
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala
index c1b92699c6e..3b99738c78b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala
@@ -170,7 +170,7 @@ trait DescribeTableSuiteBase extends QueryTest with DDLCommandTestUtils {
       val errMsg = intercept[AnalysisException] {
         sql(s"DESCRIBE TABLE $tbl col.x")
       }.getMessage
-      assert(errMsg === "DESC TABLE COLUMN does not support nested column: col.x")
+      assert(errMsg === "DESC TABLE COLUMN does not support nested column: col.x.")
     }
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala
index da4eab13afb..fe017866f25 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DescribeTableSuite.scala
@@ -59,7 +59,7 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase
       val errMsg = intercept[AnalysisException] {
         sql(s"DESC $tbl key1").collect()
       }.getMessage
-      assert(errMsg === "Column key1 does not exist")
+      assert(errMsg === "Column key1 does not exist.")
     }
   }
 
@@ -79,7 +79,7 @@ trait DescribeTableSuiteBase extends command.DescribeTableSuiteBase
         val errMsg = intercept[AnalysisException] {
           sql(s"DESC $tbl KEY").collect()
         }.getMessage
-        assert(errMsg === "Column KEY does not exist")
+        assert(errMsg === "Column KEY does not exist.")
       }
     }
   }
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala
index 3fcc8612b89..291268380f1 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala
@@ -170,11 +170,19 @@ class HiveUDAFSuite extends QueryTest
     val functionClass = "org.apache.spark.sql.hive.execution.LongProductSum"
     withUserDefinedFunction(functionName -> true) {
       sql(s"CREATE TEMPORARY FUNCTION $functionName AS '$functionClass'")
-      val e = intercept[AnalysisException] {
-        sql(s"SELECT $functionName(100)")
-      }.getMessage
-      assert(e.contains(
-        s"Invalid number of arguments for function $functionName. Expected: 2; Found: 1;"))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"SELECT $functionName(100)")
+        },
+        errorClass = "_LEGACY_ERROR_TEMP_1042",
+        parameters = Map(
+          "name" -> "longProductSum",
+          "expectedInfo" -> "2",
+          "actualNumber" -> "1"),
+        context = ExpectedContext(
+          fragment = "longProductSum(100)",
+          start = 7,
+          stop = 25))
     }
   }
 }


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