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 295c615b16b [SPARK-44404][SQL] Assign names to the error class 
_LEGACY_ERROR_TEMP_[1009,1010,1013,1015,1016,1278]
295c615b16b is described below

commit 295c615b16b8a77f242ffa99006b4fb95f8f3487
Author: panbingkun <pbk1...@gmail.com>
AuthorDate: Sat Aug 12 12:22:28 2023 +0500

    [SPARK-44404][SQL] Assign names to the error class 
_LEGACY_ERROR_TEMP_[1009,1010,1013,1015,1016,1278]
    
    ### What changes were proposed in this pull request?
    The pr aims to assign names to the error class, include:
    - _LEGACY_ERROR_TEMP_1009 => VIEW_EXCEED_MAX_NESTED_DEPTH
    - _LEGACY_ERROR_TEMP_1010 => UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION
    - _LEGACY_ERROR_TEMP_1013 => UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION / 
UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION
    - _LEGACY_ERROR_TEMP_1014 => 
UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION
    - _LEGACY_ERROR_TEMP_1015 => UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION
    - _LEGACY_ERROR_TEMP_1016 => 
UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION
    - _LEGACY_ERROR_TEMP_1278 => UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION
    
    ### Why are the changes needed?
    The changes improve the error framework.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    - Pass GA.
    - Manually test.
    - Update UT.
    
    Closes #42109 from panbingkun/SPARK-44404.
    
    Lead-authored-by: panbingkun <pbk1...@gmail.com>
    Co-authored-by: panbingkun <84731...@qq.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 R/pkg/tests/fulltests/test_sparkSQL.R              |   3 +-
 .../src/main/resources/error/error-classes.json    |  91 ++++---
 ...ions-unsupported-table-operation-error-class.md |  36 +++
 ...-unsupported-temp-view-operation-error-class.md |  36 +++
 ...tions-unsupported-view-operation-error-class.md |  36 +++
 docs/sql-error-conditions.md                       |  30 +++
 .../spark/sql/catalyst/analysis/Analyzer.scala     |   9 +-
 .../sql/catalyst/analysis/v2ResolutionPlans.scala  |   4 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala     |  32 ++-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  90 ++++---
 .../spark/sql/catalyst/parser/DDLParserSuite.scala | 104 ++++----
 .../apache/spark/sql/execution/command/views.scala |   2 +-
 .../apache/spark/sql/internal/CatalogImpl.scala    |   2 +-
 .../analyzer-results/change-column.sql.out         |  16 +-
 .../sql-tests/results/change-column.sql.out        |  16 +-
 .../spark/sql/connector/DataSourceV2SQLSuite.scala |   7 +-
 .../apache/spark/sql/execution/SQLViewSuite.scala  | 267 ++++++++++++++-------
 .../spark/sql/execution/SQLViewTestSuite.scala     |  23 +-
 .../AlterTableAddPartitionParserSuite.scala        |   4 +-
 .../AlterTableDropPartitionParserSuite.scala       |   8 +-
 .../AlterTableRecoverPartitionsParserSuite.scala   |   8 +-
 .../AlterTableRenamePartitionParserSuite.scala     |   4 +-
 .../command/AlterTableSetLocationParserSuite.scala |   6 +-
 .../command/AlterTableSetSerdeParserSuite.scala    |  16 +-
 .../spark/sql/execution/command/DDLSuite.scala     |  36 ++-
 .../command/MsckRepairTableParserSuite.scala       |  13 +-
 .../command/ShowPartitionsParserSuite.scala        |  10 +-
 .../command/TruncateTableParserSuite.scala         |   6 +-
 .../execution/command/TruncateTableSuiteBase.scala |  45 +++-
 .../execution/command/v1/ShowPartitionsSuite.scala |  57 ++++-
 .../apache/spark/sql/internal/CatalogSuite.scala   |  13 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala    |  94 +++-----
 32 files changed, 717 insertions(+), 407 deletions(-)

diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R 
b/R/pkg/tests/fulltests/test_sparkSQL.R
index d61501d248a..47688d7560c 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -4193,8 +4193,7 @@ test_that("catalog APIs, listTables, getTable, 
listColumns, listFunctions, funct
 
   # recoverPartitions does not work with temporary view
   expect_error(recoverPartitions("cars"),
-               paste("Error in recoverPartitions : analysis error - cars is a 
temp view.",
-                     "'recoverPartitions()' expects a table"), fixed = TRUE)
+               "[UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION]*`cars`*")
   expect_error(refreshTable("cars"), NA)
   expect_error(refreshByPath("/"), NA)
 
diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 133c2dd826c..08f79bcecbb 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -3394,12 +3394,63 @@
     },
     "sqlState" : "0A000"
   },
+  "UNSUPPORTED_TABLE_OPERATION" : {
+    "message" : [
+      "The table <tableName> does not support <operation>."
+    ],
+    "subClass" : {
+      "WITHOUT_SUGGESTION" : {
+        "message" : [
+          ""
+        ]
+      },
+      "WITH_SUGGESTION" : {
+        "message" : [
+          "Please use ALTER TABLE instead."
+        ]
+      }
+    }
+  },
+  "UNSUPPORTED_TEMP_VIEW_OPERATION" : {
+    "message" : [
+      "The temp view <tempViewName> does not support <operation>."
+    ],
+    "subClass" : {
+      "WITHOUT_SUGGESTION" : {
+        "message" : [
+          ""
+        ]
+      },
+      "WITH_SUGGESTION" : {
+        "message" : [
+          "Please use ALTER VIEW instead."
+        ]
+      }
+    }
+  },
   "UNSUPPORTED_TYPED_LITERAL" : {
     "message" : [
       "Literals of the type <unsupportedType> are not supported. Supported 
types are <supportedTypes>."
     ],
     "sqlState" : "0A000"
   },
+  "UNSUPPORTED_VIEW_OPERATION" : {
+    "message" : [
+      "The view <viewName> does not support <operation>."
+    ],
+    "subClass" : {
+      "WITHOUT_SUGGESTION" : {
+        "message" : [
+          ""
+        ]
+      },
+      "WITH_SUGGESTION" : {
+        "message" : [
+          "Please use ALTER VIEW instead."
+        ]
+      }
+    }
+  },
   "UNTYPED_SCALA_UDF" : {
     "message" : [
       "You're using untyped Scala UDF, which does not have the input type 
information. Spark may blindly pass null to the Scala closure with 
primitive-type argument, and the closure will see the default value of the Java 
type for the null argument, e.g. `udf((x: Int) => x, IntegerType)`, the result 
is 0 for null input. To get rid of this error, you could:",
@@ -3430,6 +3481,11 @@
     ],
     "sqlState" : "42P07"
   },
+  "VIEW_EXCEED_MAX_NESTED_DEPTH" : {
+    "message" : [
+      "The depth of view <viewName> exceeds the maximum view resolution depth 
(<maxNestedDepth>). Analysis is aborted to avoid errors. If you want to work 
around this, please try to increase the value of 
\"spark.sql.view.maxNestedViewDepth\"."
+    ]
+  },
   "VIEW_NOT_FOUND" : {
     "message" : [
       "The view <relationName> cannot be found. Verify the spelling and 
correctness of the schema and catalog.",
@@ -3702,16 +3758,6 @@
       "<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>."
@@ -3722,26 +3768,6 @@
       "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>."
@@ -4642,11 +4668,6 @@
       "The logical plan that represents the view is not analyzed."
     ]
   },
-  "_LEGACY_ERROR_TEMP_1278" : {
-    "message" : [
-      "<name> is not a view."
-    ]
-  },
   "_LEGACY_ERROR_TEMP_1280" : {
     "message" : [
       "It is not allowed to create a persisted view from the Dataset API."
diff --git 
a/docs/sql-error-conditions-unsupported-table-operation-error-class.md 
b/docs/sql-error-conditions-unsupported-table-operation-error-class.md
new file mode 100644
index 00000000000..5e4c07ccffd
--- /dev/null
+++ b/docs/sql-error-conditions-unsupported-table-operation-error-class.md
@@ -0,0 +1,36 @@
+---
+layout: global
+title: UNSUPPORTED_TABLE_OPERATION error class
+displayTitle: UNSUPPORTED_TABLE_OPERATION error class
+license: |
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+---
+
+SQLSTATE: none assigned
+
+The table `<tableName>` does not support `<operation>`.
+
+This error class has the following derived error classes:
+
+## WITHOUT_SUGGESTION
+
+
+
+## WITH_SUGGESTION
+
+Please use ALTER TABLE instead.
+
+
diff --git 
a/docs/sql-error-conditions-unsupported-temp-view-operation-error-class.md 
b/docs/sql-error-conditions-unsupported-temp-view-operation-error-class.md
new file mode 100644
index 00000000000..bb156789e85
--- /dev/null
+++ b/docs/sql-error-conditions-unsupported-temp-view-operation-error-class.md
@@ -0,0 +1,36 @@
+---
+layout: global
+title: UNSUPPORTED_TEMP_VIEW_OPERATION error class
+displayTitle: UNSUPPORTED_TEMP_VIEW_OPERATION error class
+license: |
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+---
+
+SQLSTATE: none assigned
+
+The temp view `<tempViewName>` does not support `<operation>`.
+
+This error class has the following derived error classes:
+
+## WITHOUT_SUGGESTION
+
+
+
+## WITH_SUGGESTION
+
+Please use ALTER VIEW instead.
+
+
diff --git 
a/docs/sql-error-conditions-unsupported-view-operation-error-class.md 
b/docs/sql-error-conditions-unsupported-view-operation-error-class.md
new file mode 100644
index 00000000000..88f3adc3fdb
--- /dev/null
+++ b/docs/sql-error-conditions-unsupported-view-operation-error-class.md
@@ -0,0 +1,36 @@
+---
+layout: global
+title: UNSUPPORTED_VIEW_OPERATION error class
+displayTitle: UNSUPPORTED_VIEW_OPERATION error class
+license: |
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+---
+
+SQLSTATE: none assigned
+
+The view `<viewName>` does not support `<operation>`.
+
+This error class has the following derived error classes:
+
+## WITHOUT_SUGGESTION
+
+
+
+## WITH_SUGGESTION
+
+Please use ALTER VIEW instead.
+
+
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index bd49ba94f5f..b099560184e 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -2080,12 +2080,36 @@ Unsupported subquery expression:
 
 For more details see 
[UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY](sql-error-conditions-unsupported-subquery-expression-category-error-class.html)
 
+### 
[UNSUPPORTED_TABLE_OPERATION](sql-error-conditions-unsupported-table-operation-error-class.html)
+
+SQLSTATE: none assigned
+
+The table `<tableName>` does not support `<operation>`.
+
+For more details see 
[UNSUPPORTED_TABLE_OPERATION](sql-error-conditions-unsupported-table-operation-error-class.html)
+
+### 
[UNSUPPORTED_TEMP_VIEW_OPERATION](sql-error-conditions-unsupported-temp-view-operation-error-class.html)
+
+SQLSTATE: none assigned
+
+The temp view `<tempViewName>` does not support `<operation>`.
+
+For more details see 
[UNSUPPORTED_TEMP_VIEW_OPERATION](sql-error-conditions-unsupported-temp-view-operation-error-class.html)
+
 ### UNSUPPORTED_TYPED_LITERAL
 
 [SQLSTATE: 
0A000](sql-error-conditions-sqlstates.html#class-0A-feature-not-supported)
 
 Literals of the type `<unsupportedType>` are not supported. Supported types 
are `<supportedTypes>`.
 
+### 
[UNSUPPORTED_VIEW_OPERATION](sql-error-conditions-unsupported-view-operation-error-class.html)
+
+SQLSTATE: none assigned
+
+The view `<viewName>` does not support `<operation>`.
+
+For more details see 
[UNSUPPORTED_VIEW_OPERATION](sql-error-conditions-unsupported-view-operation-error-class.html)
+
 ### UNTYPED_SCALA_UDF
 
 SQLSTATE: none assigned
@@ -2117,6 +2141,12 @@ To tolerate the error on drop use DROP VARIABLE IF 
EXISTS.
 Cannot create view `<relationName>` because it already exists.
 Choose a different name, drop or replace the existing object, or add the IF 
NOT EXISTS clause to tolerate pre-existing objects.
 
+### VIEW_EXCEED_MAX_NESTED_DEPTH
+
+SQLSTATE: none assigned
+
+The depth of view `<viewName>` exceeds the maximum view resolution depth 
(`<maxNestedDepth>`). Analysis is aborted to avoid errors. If you want to work 
around this, please try to increase the value of 
"spark.sql.view.maxNestedViewDepth".
+
 ### VIEW_NOT_FOUND
 
 [SQLSTATE: 
42P01](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index e4bf5f44ea2..9c3d7ed432e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -1134,20 +1134,21 @@ class Analyzer(override val catalogManager: 
CatalogManager) extends RuleExecutor
           case v: ResolvedPersistentView =>
             val nameParts = v.catalog.name() +: 
v.identifier.asMultipartIdentifier
             throw QueryCompilationErrors.expectTableNotViewError(
-              nameParts, isTemp = false, cmd, relationTypeMismatchHint, u)
+              nameParts, isTemp = false, cmd, true, u)
           case _: ResolvedTempView =>
             throw QueryCompilationErrors.expectTableNotViewError(
-              identifier, isTemp = true, cmd, relationTypeMismatchHint, u)
+              identifier, isTemp = true, cmd, true, u)
           case table => table
         }.getOrElse(u)
 
-      case u @ UnresolvedView(identifier, cmd, allowTemp, 
relationTypeMismatchHint) =>
+      case u @ UnresolvedView(identifier, cmd, allowTemp, hint) =>
         lookupTableOrView(identifier, viewOnly = true).map {
           case _: ResolvedTempView if !allowTemp =>
             throw 
QueryCompilationErrors.expectViewNotTempViewError(identifier, cmd, u)
           case t: ResolvedTable =>
+            val nameParts = t.catalog.name() +: 
t.identifier.asMultipartIdentifier
             throw QueryCompilationErrors.expectViewNotTableError(
-              t, cmd, relationTypeMismatchHint, u)
+              nameParts, cmd, hint, u)
           case other => other
         }.getOrElse(u)
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
index 15ece1226d8..1deeef0d37e 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/v2ResolutionPlans.scala
@@ -44,7 +44,7 @@ case class UnresolvedNamespace(multipartIdentifier: 
Seq[String]) extends Unresol
 case class UnresolvedTable(
     multipartIdentifier: Seq[String],
     commandName: String,
-    relationTypeMismatchHint: Option[String]) extends UnresolvedLeafNode
+    hint: Boolean = false) extends UnresolvedLeafNode
 
 /**
  * Holds the name of a view that has yet to be looked up. It will be resolved 
to
@@ -54,7 +54,7 @@ case class UnresolvedView(
     multipartIdentifier: Seq[String],
     commandName: String,
     allowTemp: Boolean,
-    relationTypeMismatchHint: Option[String]) extends UnresolvedLeafNode
+    hint: Boolean) extends UnresolvedLeafNode
 
 /**
  * Holds the name of a table or view that has yet to be looked up in a 
catalog. It will
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 27b88197f86..b6b205f8876 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -2812,8 +2812,8 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
   private def createUnresolvedTable(
       ctx: IdentifierReferenceContext,
       commandName: String,
-      relationTypeMismatchHint: Option[String] = None): LogicalPlan = 
withOrigin(ctx) {
-    withIdentClause(ctx, UnresolvedTable(_, commandName, 
relationTypeMismatchHint))
+      hint: Boolean = false): LogicalPlan = withOrigin(ctx) {
+    withIdentClause(ctx, UnresolvedTable(_, commandName, hint))
   }
 
   /**
@@ -2823,8 +2823,8 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       ctx: IdentifierReferenceContext,
       commandName: String,
       allowTemp: Boolean = true,
-      relationTypeMismatchHint: Option[String] = None): LogicalPlan = 
withOrigin(ctx) {
-    withIdentClause(ctx, UnresolvedView(_, commandName, allowTemp, 
relationTypeMismatchHint))
+      hint: Boolean = false): LogicalPlan = withOrigin(ctx) {
+    withIdentClause(ctx, UnresolvedView(_, commandName, allowTemp, hint))
   }
 
   /**
@@ -4359,14 +4359,14 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
           ctx.identifierReference,
           commandName = "ALTER VIEW ... SET TBLPROPERTIES",
           allowTemp = false,
-          relationTypeMismatchHint = alterViewTypeMismatchHint),
+          hint = true),
         cleanedTableProperties)
     } else {
       SetTableProperties(
         createUnresolvedTable(
           ctx.identifierReference,
           "ALTER TABLE ... SET TBLPROPERTIES",
-          alterTableTypeMismatchHint),
+          true),
         cleanedTableProperties)
     }
   }
@@ -4392,7 +4392,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
           ctx.identifierReference,
           commandName = "ALTER VIEW ... UNSET TBLPROPERTIES",
           allowTemp = false,
-          relationTypeMismatchHint = alterViewTypeMismatchHint),
+          hint = true),
         cleanedProperties,
         ifExists)
     } else {
@@ -4400,7 +4400,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
         createUnresolvedTable(
           ctx.identifierReference,
           "ALTER TABLE ... UNSET TBLPROPERTIES",
-          alterTableTypeMismatchHint),
+          true),
         cleanedProperties,
         ifExists)
     }
@@ -4419,7 +4419,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... SET LOCATION ...",
-        alterTableTypeMismatchHint),
+        true),
       Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec),
       visitLocationSpec(ctx.locationSpec))
   }
@@ -4716,7 +4716,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... RECOVER PARTITIONS",
-        alterTableTypeMismatchHint))
+        true))
   }
 
   /**
@@ -4746,7 +4746,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... ADD PARTITION ...",
-        alterTableTypeMismatchHint),
+        true),
       specsAndLocs.toSeq,
       ctx.EXISTS != null)
   }
@@ -4765,7 +4765,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... RENAME TO PARTITION",
-        alterTableTypeMismatchHint),
+        true),
       UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)),
       UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.to)))
   }
@@ -4794,7 +4794,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... DROP PARTITION ...",
-        alterTableTypeMismatchHint),
+        true),
       partSpecs.toSeq,
       ifExists = ctx.EXISTS != null,
       purge = ctx.PURGE != null)
@@ -4815,7 +4815,7 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       createUnresolvedTable(
         ctx.identifierReference,
         "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]",
-        alterTableTypeMismatchHint),
+        true),
       Option(ctx.stringLit).map(x => string(visitStringLit(x))),
       Option(ctx.propertyList).map(visitPropertyKeyValues),
       // TODO a partition spec is allowed to have optional values. This is 
currently violated.
@@ -5008,10 +5008,6 @@ class AstBuilder extends DataTypeAstBuilder with 
SQLConfHelper with Logging {
       ctx.EXISTS != null)
   }
 
-  private def alterViewTypeMismatchHint: Option[String] = Some("Please use 
ALTER TABLE instead.")
-
-  private def alterTableTypeMismatchHint: Option[String] = Some("Please use 
ALTER VIEW instead.")
-
   /**
    * Create a TimestampAdd expression.
    */
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 5b3c3daa75b..2ef9c15a7f9 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
@@ -432,20 +432,21 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
   }
 
   def viewDepthExceedsMaxResolutionDepthError(
-      identifier: TableIdentifier, maxNestedViewDepth: Int, t: TreeNode[_]): 
Throwable = {
+      identifier: TableIdentifier, maxNestedDepth: Int, t: TreeNode[_]): 
Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1009",
+      errorClass = "VIEW_EXCEED_MAX_NESTED_DEPTH",
       messageParameters = Map(
-        "identifier" -> identifier.toString,
-        "maxNestedViewDepth" -> maxNestedViewDepth.toString,
-        "config" -> SQLConf.MAX_NESTED_VIEW_DEPTH.key),
+        "viewName" -> toSQLId(identifier.nameParts),
+        "maxNestedDepth" -> maxNestedDepth.toString),
       origin = t.origin)
   }
 
   def insertIntoViewNotAllowedError(identifier: TableIdentifier, t: 
TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1010",
-      messageParameters = Map("identifier" -> identifier.toString),
+      errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+      messageParameters = Map(
+        "viewName" -> toSQLId(identifier.nameParts),
+        "operation" -> "INSERT"),
       origin = t.origin)
   }
 
@@ -467,18 +468,31 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
       nameParts: Seq[String],
       isTemp: Boolean,
       cmd: String,
-      mismatchHint: Option[String],
+      hint: Boolean,
       t: TreeNode[_]): Throwable = {
-    val viewStr = if (isTemp) "temp view" else "view"
-    val hintStr = mismatchHint.map(" " + _).getOrElse("")
-    new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1013",
-      messageParameters = Map(
-        "nameParts" -> nameParts.quoted,
-        "viewStr" -> viewStr,
-        "cmd" -> cmd,
-        "hintStr" -> hintStr),
-      origin = t.origin)
+    if (isTemp) {
+      new AnalysisException(
+        errorClass = if (hint) {
+          "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION"
+        } else {
+          "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION"
+        },
+        messageParameters = Map(
+          "tempViewName" -> toSQLId(nameParts),
+          "operation" -> cmd),
+        origin = t.origin)
+    } else {
+      new AnalysisException(
+        errorClass = if (hint) {
+          "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION"
+        } else {
+          "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION"
+        },
+        messageParameters = Map(
+          "viewName" -> toSQLId(nameParts),
+          "operation" -> cmd),
+        origin = t.origin)
+    }
   }
 
   def expectViewNotTempViewError(
@@ -486,32 +500,37 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
       cmd: String,
       t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1014",
+      errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
       messageParameters = Map(
-        "nameParts" -> nameParts.quoted,
-        "cmd" -> cmd),
+        "tempViewName" -> toSQLId(nameParts),
+        "operation" -> cmd),
       origin = t.origin)
   }
 
   def expectViewNotTableError(
-      v: ResolvedTable, cmd: String, mismatchHint: Option[String], t: 
TreeNode[_]): Throwable = {
-    val hintStr = mismatchHint.map(" " + _).getOrElse("")
+      nameParts: Seq[String],
+      cmd: String,
+      hint: Boolean,
+      t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1015",
+      errorClass = if (hint) {
+        "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION"
+      } else {
+        "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION"
+      },
       messageParameters = Map(
-        "identifier" -> v.identifier.quoted,
-        "cmd" -> cmd,
-        "hintStr" -> hintStr),
+        "tableName" -> toSQLId(nameParts),
+        "operation" -> cmd),
       origin = t.origin)
   }
 
   def expectTableOrPermanentViewNotTempViewError(
       nameParts: Seq[String], cmd: String, t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1016",
+      errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
       messageParameters = Map(
-        "nameParts" -> nameParts.quoted,
-        "cmd" -> cmd),
+        "tempViewName" -> toSQLId(nameParts),
+        "operation" -> cmd),
       origin = t.origin)
   }
 
@@ -2841,10 +2860,15 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
         "dataColumns" -> query.output.map(c => toSQLId(c.name)).mkString(", 
")))
   }
 
-  def tableIsNotViewError(name: TableIdentifier): Throwable = {
+  def tableIsNotViewError(name: TableIdentifier, replace: Boolean): Throwable 
= {
+    val operation = if (replace) "CREATE OR REPLACE VIEW" else "CREATE VIEW"
     new AnalysisException(
-      errorClass = "_LEGACY_ERROR_TEMP_1278",
-      messageParameters = Map("name" -> name.toString))
+      errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+      messageParameters = Map(
+        "tableName" -> toSQLId(name.nameParts),
+        "operation" -> operation
+      )
+    )
   }
 
   def viewAlreadyExistsError(name: TableIdentifier): Throwable = {
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 31fd232181a..9644f6ea038 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -992,20 +992,19 @@ class DDLParserSuite extends AnalysisTest {
         "'comment' = 'new_comment')"
     val sql2_view = "ALTER VIEW table_name UNSET TBLPROPERTIES ('comment', 
'test')"
     val sql3_view = "ALTER VIEW table_name UNSET TBLPROPERTIES IF EXISTS 
('comment', 'test')"
-    val hint = Some("Please use ALTER TABLE instead.")
 
     comparePlans(parsePlan(sql1_view),
       SetViewProperties(
-        UnresolvedView(Seq("table_name"), "ALTER VIEW ... SET TBLPROPERTIES", 
false, hint),
+        UnresolvedView(Seq("table_name"), "ALTER VIEW ... SET TBLPROPERTIES", 
false, true),
         Map("test" -> "test", "comment" -> "new_comment")))
     comparePlans(parsePlan(sql2_view),
       UnsetViewProperties(
-        UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET 
TBLPROPERTIES", false, hint),
+        UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET 
TBLPROPERTIES", false, true),
         Seq("comment", "test"),
         ifExists = false))
     comparePlans(parsePlan(sql3_view),
       UnsetViewProperties(
-        UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET 
TBLPROPERTIES", false, hint),
+        UnresolvedView(Seq("table_name"), "ALTER VIEW ... UNSET 
TBLPROPERTIES", false, true),
         Seq("comment", "test"),
         ifExists = true))
   }
@@ -1017,23 +1016,22 @@ class DDLParserSuite extends AnalysisTest {
         "'comment' = 'new_comment')"
     val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 
'test')"
     val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS 
('comment', 'test')"
-    val hint = Some("Please use ALTER VIEW instead.")
 
     comparePlans(
       parsePlan(sql1_table),
       SetTableProperties(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
TBLPROPERTIES", hint),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
TBLPROPERTIES", true),
         Map("test" -> "test", "comment" -> "new_comment")))
     comparePlans(
       parsePlan(sql2_table),
       UnsetTableProperties(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET 
TBLPROPERTIES", hint),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET 
TBLPROPERTIES", true),
         Seq("comment", "test"),
         ifExists = false))
     comparePlans(
       parsePlan(sql3_table),
       UnsetTableProperties(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET 
TBLPROPERTIES", hint),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET 
TBLPROPERTIES", true),
         Seq("comment", "test"),
         ifExists = true))
   }
@@ -1042,7 +1040,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x int"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None)
       )))
   }
@@ -1051,7 +1049,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS"),
         Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None),
           QualifiedColType(None, "y", StringType, true, None, None, None)
       )))
@@ -1061,7 +1059,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMNS x int"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS"),
         Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None)
       )))
   }
@@ -1070,7 +1068,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS"),
         Seq(QualifiedColType(None, "x", IntegerType, true, None, None, None)
       )))
   }
@@ -1079,7 +1077,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMNS"),
         Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, 
None)
       )))
   }
@@ -1088,7 +1086,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x int NOT NULL"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(None, "x", IntegerType, false, None, None, None)
       )))
   }
@@ -1097,7 +1095,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(None, "x", IntegerType, true, Some("doc"), None, 
None)
       )))
   }
@@ -1106,7 +1104,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x int FIRST"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(
           None,
           "x",
@@ -1120,7 +1118,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x int AFTER y"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(
           None,
           "x",
@@ -1136,7 +1134,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc'"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(
           Some(UnresolvedFieldName(Seq("x", "y"))), "z", IntegerType, true, 
Some("doc"), None, None)
       )))
@@ -1146,7 +1144,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', 
a.b string FIRST"),
       AddColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN", None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ADD COLUMN"),
         Seq(
           QualifiedColType(
             Some(UnresolvedFieldName(Seq("x", "y"))),
@@ -1171,7 +1169,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"),
       RenameColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         "d"))
   }
@@ -1180,7 +1178,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         Some(LongType),
         None,
@@ -1206,7 +1204,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         Some(LongType),
         None,
@@ -1219,7 +1217,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c COMMENT 'new 
comment'"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         None,
@@ -1232,7 +1230,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c FIRST"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         None,
@@ -1265,7 +1263,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c SET NOT NULL"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         Some(false),
@@ -1276,7 +1274,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c DROP NOT NULL"),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         Some(true),
@@ -1289,14 +1287,14 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"),
       DropColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"),
         Seq(UnresolvedFieldName(Seq("a", "b", "c"))),
         ifExists = false))
 
     comparePlans(
       parsePlan("ALTER TABLE table_name DROP COLUMN IF EXISTS a.b.c"),
       DropColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"),
         Seq(UnresolvedFieldName(Seq("a", "b", "c"))),
         ifExists = true))
   }
@@ -1307,7 +1305,7 @@ class DDLParserSuite extends AnalysisTest {
       comparePlans(
         parsePlan(drop),
         DropColumns(
-          UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS", 
None),
+          UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"),
           Seq(UnresolvedFieldName(Seq("x")),
             UnresolvedFieldName(Seq("y")),
             UnresolvedFieldName(Seq("a", "b", "c"))),
@@ -1319,7 +1317,7 @@ class DDLParserSuite extends AnalysisTest {
       comparePlans(
         parsePlan(drop),
         DropColumns(
-          UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS", 
None),
+          UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"),
           Seq(UnresolvedFieldName(Seq("x")),
             UnresolvedFieldName(Seq("y")),
             UnresolvedFieldName(Seq("a", "b", "c"))),
@@ -1335,7 +1333,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan(sql1),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         Some(IntegerType),
         None,
@@ -1346,7 +1344,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan(sql2),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         Some(IntegerType),
         None,
@@ -1357,7 +1355,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan(sql3),
       AlterColumn(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... CHANGE COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         Some(IntegerType),
         None,
@@ -1400,19 +1398,19 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan(sql1),
       ReplaceColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS"),
         Seq(QualifiedColType(None, "x", StringType, true, None, None, None))))
 
     comparePlans(
       parsePlan(sql2),
       ReplaceColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS"),
         Seq(QualifiedColType(None, "x", StringType, true, Some("x1"), None, 
None))))
 
     comparePlans(
       parsePlan(sql3),
       ReplaceColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS"),
         Seq(
           QualifiedColType(None, "x", StringType, true, Some("x1"), None, 
None),
           QualifiedColType(None, "y", IntegerType, true, None, None, None)
@@ -1421,7 +1419,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan(sql4),
       ReplaceColumns(
-        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS", 
None),
+        UnresolvedTable(Seq("table_name"), "ALTER TABLE ... REPLACE COLUMNS"),
         Seq(
           QualifiedColType(None, "x", StringType, true, Some("x1"), None, 
None),
           QualifiedColType(None, "y", IntegerType, true, Some("y1"), None, 
None)
@@ -2279,7 +2277,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("LOAD DATA INPATH 'filepath' INTO TABLE a.b.c"),
       LoadData(
-        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
+        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
         "filepath",
         false,
         false,
@@ -2288,7 +2286,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("LOAD DATA LOCAL INPATH 'filepath' INTO TABLE a.b.c"),
       LoadData(
-        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
+        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
         "filepath",
         true,
         false,
@@ -2297,7 +2295,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("LOAD DATA LOCAL INPATH 'filepath' OVERWRITE INTO TABLE 
a.b.c"),
       LoadData(
-        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
+        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
         "filepath",
         true,
         true,
@@ -2310,7 +2308,7 @@ class DDLParserSuite extends AnalysisTest {
            |PARTITION(ds='2017-06-10')
          """.stripMargin),
       LoadData(
-        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA", None),
+        UnresolvedTable(Seq("a", "b", "c"), "LOAD DATA"),
         "filepath",
         true,
         true,
@@ -2413,7 +2411,7 @@ class DDLParserSuite extends AnalysisTest {
   test("alter view: AS Query") {
     val parsed = parsePlan("ALTER VIEW a.b.c AS SELECT 1")
     val expected = AlterViewAs(
-      UnresolvedView(Seq("a", "b", "c"), "ALTER VIEW ... AS", true, None),
+      UnresolvedView(Seq("a", "b", "c"), "ALTER VIEW ... AS", true, false),
       "SELECT 1",
       parsePlan("SELECT 1"))
     comparePlans(parsed, expected)
@@ -2451,28 +2449,28 @@ class DDLParserSuite extends AnalysisTest {
 
   test("CREATE INDEX") {
     parseCompare("CREATE index i1 ON a.b.c USING BTREE (col1)",
-      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX", None), 
"i1", "BTREE", false,
+      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX"), "i1", 
"BTREE", false,
         Seq(UnresolvedFieldName(Seq("col1"))).zip(Seq(Map.empty[String, 
String])), Map.empty))
 
     parseCompare("CREATE index IF NOT EXISTS i1 ON TABLE a.b.c USING BTREE" +
       " (col1 OPTIONS ('k1'='v1'), col2 OPTIONS ('k2'='v2')) ",
-      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX", None), 
"i1", "BTREE", true,
+      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX"), "i1", 
"BTREE", true,
         Seq(UnresolvedFieldName(Seq("col1")), UnresolvedFieldName(Seq("col2")))
           .zip(Seq(Map("k1" -> "v1"), Map("k2" -> "v2"))), Map.empty))
 
     parseCompare("CREATE index i1 ON a.b.c" +
       " (col1 OPTIONS ('k1'='v1'), col2 OPTIONS ('k2'='v2')) OPTIONS 
('k3'='v3', 'k4'='v4')",
-      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX", None), 
"i1", "", false,
+      CreateIndex(UnresolvedTable(Seq("a", "b", "c"), "CREATE INDEX"), "i1", 
"", false,
         Seq(UnresolvedFieldName(Seq("col1")), UnresolvedFieldName(Seq("col2")))
           .zip(Seq(Map("k1" -> "v1"), Map("k2" -> "v2"))), Map("k3" -> "v3", 
"k4" -> "v4")))
   }
 
   test("DROP INDEX") {
     parseCompare("DROP index i1 ON a.b.c",
-      DropIndex(UnresolvedTable(Seq("a", "b", "c"), "DROP INDEX", None), "i1", 
false))
+      DropIndex(UnresolvedTable(Seq("a", "b", "c"), "DROP INDEX"), "i1", 
false))
 
     parseCompare("DROP index IF EXISTS i1 ON a.b.c",
-      DropIndex(UnresolvedTable(Seq("a", "b", "c"), "DROP INDEX", None), "i1", 
true))
+      DropIndex(UnresolvedTable(Seq("a", "b", "c"), "DROP INDEX"), "i1", true))
   }
 
   private case class TableSpec(
@@ -2562,7 +2560,7 @@ class DDLParserSuite extends AnalysisTest {
 
     comparePlans(
       parsePlan("COMMENT ON TABLE a.b.c IS 'xYz'"),
-      CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE", 
None), "xYz"))
+      CommentOnTable(UnresolvedTable(Seq("a", "b", "c"), "COMMENT ON TABLE"), 
"xYz"))
   }
 
   test("create table - without using") {
@@ -2641,12 +2639,12 @@ class DDLParserSuite extends AnalysisTest {
     // These ALTER TABLE statements should parse successfully.
     comparePlans(
       parsePlan("ALTER TABLE t1 ADD COLUMN x int NOT NULL DEFAULT 42"),
-      AddColumns(UnresolvedTable(Seq("t1"), "ALTER TABLE ... ADD COLUMN", 
None),
+      AddColumns(UnresolvedTable(Seq("t1"), "ALTER TABLE ... ADD COLUMN"),
         Seq(QualifiedColType(None, "x", IntegerType, false, None, None, 
Some("42")))))
     comparePlans(
       parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT 42"),
       AlterColumn(
-        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN", None),
+        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         None,
@@ -2657,7 +2655,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c SET DEFAULT ''"),
       AlterColumn(
-        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN", None),
+        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         None,
@@ -2682,7 +2680,7 @@ class DDLParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("ALTER TABLE t1 ALTER COLUMN a.b.c DROP DEFAULT"),
       AlterColumn(
-        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN", None),
+        UnresolvedTable(Seq("t1"), "ALTER TABLE ... ALTER COLUMN"),
         UnresolvedFieldName(Seq("a", "b", "c")),
         None,
         None,
@@ -2830,7 +2828,7 @@ class DDLParserSuite extends AnalysisTest {
   test("SPARK-42681: Relax ordering constraint for ALTER TABLE ADD COLUMN 
options") {
     // Positive test cases to verify that column definition options could be 
applied in any order.
     val expectedPlan = AddColumns(
-      UnresolvedTable(Seq("my_tab"), "ALTER TABLE ... ADD COLUMN", None),
+      UnresolvedTable(Seq("my_tab"), "ALTER TABLE ... ADD COLUMN"),
       Seq(
         QualifiedColType(
           path = None,
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
index 7b95d34e6b6..b27bd5bf909 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala
@@ -146,7 +146,7 @@ case class CreateViewCommand(
         // Handles `CREATE VIEW IF NOT EXISTS v0 AS SELECT ...`. Does nothing 
when the target view
         // already exists.
       } else if (tableMetadata.tableType != CatalogTableType.VIEW) {
-        throw QueryCompilationErrors.tableIsNotViewError(name)
+        throw QueryCompilationErrors.tableIsNotViewError(name, replace)
       } else if (replace) {
         // Detect cyclic view reference on CREATE OR REPLACE VIEW.
         val viewIdent = tableMetadata.identifier
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 796710a3567..acc2055d779 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -746,7 +746,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
     val multiPartIdent = 
sparkSession.sessionState.sqlParser.parseMultipartIdentifier(tableName)
     sparkSession.sessionState.executePlan(
       RecoverPartitions(
-        UnresolvedTable(multiPartIdent, "recoverPartitions()", None))).toRdd
+        UnresolvedTable(multiPartIdent, "recoverPartitions()"))).toRdd
   }
 
   /**
diff --git 
a/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out 
b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out
index 94dcd99bbd6..fe78fbeef60 100644
--- 
a/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out
+++ 
b/sql/core/src/test/resources/sql-tests/analyzer-results/change-column.sql.out
@@ -206,12 +206,10 @@ ALTER TABLE temp_view CHANGE a TYPE INT
 -- !query analysis
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "errorClass" : "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
   "messageParameters" : {
-    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
-    "hintStr" : "",
-    "nameParts" : "temp_view",
-    "viewStr" : "temp view"
+    "operation" : "ALTER TABLE ... CHANGE COLUMN",
+    "tempViewName" : "`temp_view`"
   },
   "queryContext" : [ {
     "objectType" : "",
@@ -236,12 +234,10 @@ ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT
 -- !query analysis
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "errorClass" : "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
   "messageParameters" : {
-    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
-    "hintStr" : "",
-    "nameParts" : "global_temp.global_temp_view",
-    "viewStr" : "temp view"
+    "operation" : "ALTER TABLE ... CHANGE COLUMN",
+    "tempViewName" : "`global_temp`.`global_temp_view`"
   },
   "queryContext" : [ {
     "objectType" : "",
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 5937f52c468..41192f7fd5d 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
@@ -270,12 +270,10 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "errorClass" : "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
   "messageParameters" : {
-    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
-    "hintStr" : "",
-    "nameParts" : "temp_view",
-    "viewStr" : "temp view"
+    "operation" : "ALTER TABLE ... CHANGE COLUMN",
+    "tempViewName" : "`temp_view`"
   },
   "queryContext" : [ {
     "objectType" : "",
@@ -302,12 +300,10 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1013",
+  "errorClass" : "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
   "messageParameters" : {
-    "cmd" : "ALTER TABLE ... CHANGE COLUMN",
-    "hintStr" : "",
-    "nameParts" : "global_temp.global_temp_view",
-    "viewStr" : "temp view"
+    "operation" : "ALTER TABLE ... CHANGE COLUMN",
+    "tempViewName" : "`global_temp`.`global_temp_view`"
   },
   "queryContext" : [ {
     "objectType" : "",
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 06f5600e0d1..fa4f37628e0 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
@@ -2601,11 +2601,10 @@ class DataSourceV2SQLSuiteV1Filter
       sql("create global temp view v as select 1")
       checkError(
         exception = intercept[AnalysisException](sql("COMMENT ON TABLE 
global_temp.v IS NULL")),
-        errorClass = "_LEGACY_ERROR_TEMP_1013",
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
         parameters = Map(
-          "nameParts" -> "global_temp.v",
-          "viewStr" -> "temp view",
-          "cmd" -> "COMMENT ON TABLE", "hintStr" -> ""),
+          "tempViewName" -> "`global_temp`.`v`",
+          "operation" -> "COMMENT ON TABLE"),
         context = ExpectedContext(fragment = "global_temp.v", start = 17, stop 
= 29))
     }
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
index e258d600a2a..0ab292ee6c3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala
@@ -115,25 +115,28 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1278",
-        parameters = Map("name" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`")
+        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+        parameters = Map(
+          "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
+          "operation" -> "CREATE OR REPLACE VIEW")
       )
       checkError(
         exception = intercept[AnalysisException] {
           sql("CREATE VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1278",
-        parameters = Map("name" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`")
+        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+        parameters = Map(
+          "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
+          "operation" -> "CREATE VIEW")
       )
       checkError(
         exception = intercept[AnalysisException] {
           sql("ALTER VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1015",
+        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
         parameters = Map(
-          "identifier" -> "default.tab1",
-          "cmd" -> "ALTER VIEW ... AS",
-          "hintStr" -> ""
+          "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
+          "operation" -> "ALTER VIEW ... AS"
         ),
         context = ExpectedContext(
           fragment = "tab1",
@@ -155,12 +158,36 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
     val viewName = "testView"
     withTempView(viewName) {
       spark.range(10).createTempView(viewName)
-      assertAnalysisError(
-        s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')",
-        "testView is a temp view. 'ALTER VIEW ... SET TBLPROPERTIES' expects a 
permanent view.")
-      assertAnalysisError(
-        s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')",
-        "testView is a temp view. 'ALTER VIEW ... UNSET TBLPROPERTIES' expects 
a permanent view.")
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER VIEW ... SET TBLPROPERTIES"
+        ),
+        context = ExpectedContext(
+          fragment = "testView",
+          start = 11,
+          stop = 18
+        )
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES"
+        ),
+        context = ExpectedContext(
+          fragment = "testView",
+          start = 11,
+          stop = 18
+        )
+      )
     }
   }
 
@@ -168,50 +195,127 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
     val viewName = "testView"
     withTempView(viewName) {
       spark.range(10).createTempView(viewName)
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName SET SERDE 'whatever'",
-        viewName,
-        "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'",
-        viewName,
-        "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')",
-        viewName,
-        "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')",
-        viewName,
-        "ALTER TABLE ... RENAME TO PARTITION")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName RECOVER PARTITIONS",
-        viewName,
-        "ALTER TABLE ... RECOVER PARTITIONS")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'",
-        viewName,
-        "ALTER TABLE ... SET LOCATION ...")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION 
'/path/to/home'",
-        viewName,
-        "ALTER TABLE ... SET LOCATION ...")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')",
-        viewName,
-        "ALTER TABLE ... ADD PARTITION ...")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')",
-        viewName,
-        "ALTER TABLE ... DROP PARTITION ...")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName SET TBLPROPERTIES ('p' = 'an')",
-        viewName,
-        "ALTER TABLE ... SET TBLPROPERTIES")
-      assertErrorForAlterTableOnTempView(
-        s"ALTER TABLE $viewName UNSET TBLPROPERTIES ('p')",
-        viewName,
-        "ALTER TABLE ... UNSET TBLPROPERTIES")
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName SET SERDE 'whatever'")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 
'whatever'")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION 
(a='5')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... RENAME TO PARTITION"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName RECOVER PARTITIONS")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... RECOVER PARTITIONS"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName SET LOCATION 
'/path/to/your/lovely/heart'")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET LOCATION ..."
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION 
'/path/to/home'")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET LOCATION ..."
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', 
b='8')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... ADD PARTITION ..."
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... DROP PARTITION ..."
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName SET TBLPROPERTIES ('p' = 'an')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... SET TBLPROPERTIES"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"ALTER TABLE $viewName UNSET TBLPROPERTIES ('p')")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES"
+        ),
+        context = ExpectedContext(fragment = viewName, start = 12, stop = 19)
+      )
     }
   }
 
@@ -235,12 +339,10 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(sqlText)
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1013",
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
         parameters = Map(
-          "nameParts" -> viewName,
-          "viewStr" -> "temp view",
-          "cmd" -> "LOAD DATA",
-          "hintStr" -> ""
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "LOAD DATA"
         ),
         context = ExpectedContext(
           fragment = viewName,
@@ -252,10 +354,10 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"SHOW CREATE TABLE $viewName")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1016",
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
         parameters = Map(
-          "nameParts" -> "testView",
-          "cmd" -> "SHOW CREATE TABLE"
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "SHOW CREATE TABLE"
         ),
         context = ExpectedContext(
           fragment = viewName,
@@ -267,10 +369,10 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1016",
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
         parameters = Map(
-          "nameParts" -> "testView",
-          "cmd" -> "ANALYZE TABLE"
+          "tempViewName" -> s"`$viewName`",
+          "operation" -> "ANALYZE TABLE"
         ),
         context = ExpectedContext(
           fragment = viewName,
@@ -288,11 +390,6 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
     }
   }
 
-  private def assertAnalysisError(query: String, message: String): Unit = {
-    val e = intercept[AnalysisException](sql(query))
-    assert(e.message.contains(message))
-  }
-
   private def assertAnalysisErrorClass(query: String,
       errorClass: String,
       parameters: Map[String, String],
@@ -301,13 +398,6 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
     checkError(e, errorClass = errorClass, parameters = parameters, context = 
context)
   }
 
-  private def assertErrorForAlterTableOnTempView(
-    sqlText: String, viewName: String, cmdName: String): Unit = {
-    assertAnalysisError(
-      sqlText,
-      s"$viewName is a temp view. '$cmdName' expects a table. Please use ALTER 
VIEW instead.")
-  }
-
   test("error handling: insert/load table commands against a view") {
     val viewName = "testView"
     withView(viewName) {
@@ -316,8 +406,11 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"INSERT INTO TABLE $viewName SELECT 1")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1010",
-        parameters = Map("identifier" -> 
s"`$SESSION_CATALOG_NAME`.`default`.`testview`"),
+        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        parameters = Map(
+          "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`",
+          "operation" -> "INSERT"
+        ),
         context = ExpectedContext(fragment = viewName, start = 18, stop = 25)
       )
 
@@ -328,12 +421,10 @@ abstract class SQLViewSuite extends QueryTest with 
SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(sqlText)
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1013",
+        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
         parameters = Map(
-          "nameParts" -> "spark_catalog.default.testview",
-          "viewStr" -> "view",
-          "cmd" -> "LOAD DATA",
-          "hintStr" -> ""),
+          "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`",
+          "operation" -> "LOAD DATA"),
         context = ExpectedContext(
           fragment = viewName,
           start = sqlText.length - 8,
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala
index 1bee93fa429..edd58ccceda 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewTestSuite.scala
@@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.logical.Repartition
 import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.withDefaultTimeZone
 import org.apache.spark.sql.connector.catalog._
 import 
org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME
+import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
 import org.apache.spark.sql.internal.SQLConf._
 import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils}
 import org.apache.spark.sql.types.{IntegerType, StructField, StructType}
@@ -225,13 +226,17 @@ abstract class SQLViewTestSuite extends QueryTest with 
SQLTestUtils {
           exception = intercept[AnalysisException] {
             sql(s"SELECT * FROM ${viewNames.last}")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1009",
+          errorClass = "VIEW_EXCEED_MAX_NESTED_DEPTH",
           parameters = Map(
-            "identifier" -> tableIdentifier("view0").quotedString,
-            "maxNestedViewDepth" -> "10",
-            "config" -> s"${MAX_NESTED_VIEW_DEPTH.key}"),
-          context = ExpectedContext("VIEW", 
tableIdentifier("view1").unquotedString,
-            14, 13 + formattedViewName("view0").length, 
formattedViewName("view0"))
+            "viewName" -> tableIdentifier("view0").quotedString,
+            "maxNestedDepth" -> "10"),
+          context = ExpectedContext(
+            "VIEW",
+            tableIdentifier("view1").unquotedString,
+            14,
+            13 + formattedViewName("view0").length,
+            formattedViewName("view0")
+          )
         )
       }
     }
@@ -483,10 +488,10 @@ abstract class TempViewTestSuite extends SQLViewTestSuite 
{
         exception = intercept[AnalysisException] {
           sql(s"SHOW CREATE TABLE ${formattedViewName(viewName)}")
         },
-        errorClass = "_LEGACY_ERROR_TEMP_1016",
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITHOUT_SUGGESTION",
         parameters = Map(
-          "nameParts" -> formattedViewName(viewName),
-          "cmd" -> "SHOW CREATE TABLE"),
+          "tempViewName" -> toSQLId(tableIdentifier(viewName).nameParts),
+          "operation" -> "SHOW CREATE TABLE"),
         context = ExpectedContext(
           fragment = formattedViewName(viewName),
           start = 18,
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala
index 1694c73b10f..90b1587f397 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableAddPartitionParserSuite.scala
@@ -33,7 +33,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest 
with SharedSparkSes
       UnresolvedTable(
         Seq("a", "b", "c"),
         "ALTER TABLE ... ADD PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us"), 
Some("location1")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"), 
None)),
@@ -48,7 +48,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest 
with SharedSparkSes
       UnresolvedTable(
         Seq("a", "b", "c"),
         "ALTER TABLE ... ADD PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(UnresolvedPartitionSpec(Map("dt" -> "2008-08-08"), Some("loc"))),
       ifNotExists = false)
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala
index e52c012a01b..7d6b5730c9c 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropPartitionParserSuite.scala
@@ -33,7 +33,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest 
with SharedSparkSe
       UnresolvedTable(
         Seq("table_name"),
         "ALTER TABLE ... DROP PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))),
@@ -53,7 +53,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest 
with SharedSparkSe
       UnresolvedTable(
         Seq("table_name"),
         "ALTER TABLE ... DROP PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))),
@@ -68,7 +68,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest 
with SharedSparkSe
       UnresolvedTable(
         Seq("a", "b", "c"),
         "ALTER TABLE ... DROP PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"))),
       ifExists = true,
       purge = false)
@@ -82,7 +82,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest 
with SharedSparkSe
       UnresolvedTable(
         Seq("table_name"),
         "ALTER TABLE ... DROP PARTITION ...",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       Seq(UnresolvedPartitionSpec(Map("p" -> "1"))),
       ifExists = false,
       purge = true)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
index e0a25580652..7c82b1f81ab 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
@@ -39,7 +39,7 @@ class AlterTableRecoverPartitionsParserSuite extends 
AnalysisTest with SharedSpa
         UnresolvedTable(
           Seq("tbl"),
           "ALTER TABLE ... RECOVER PARTITIONS",
-          Some("Please use ALTER VIEW instead."))))
+          true)))
   }
 
   test("recover partitions of a table in a database") {
@@ -49,7 +49,7 @@ class AlterTableRecoverPartitionsParserSuite extends 
AnalysisTest with SharedSpa
         UnresolvedTable(
           Seq("db", "tbl"),
           "ALTER TABLE ... RECOVER PARTITIONS",
-          Some("Please use ALTER VIEW instead."))))
+          true)))
   }
 
   test("recover partitions of a table spark_catalog") {
@@ -59,7 +59,7 @@ class AlterTableRecoverPartitionsParserSuite extends 
AnalysisTest with SharedSpa
         UnresolvedTable(
           Seq("spark_catalog", "db", "TBL"),
           "ALTER TABLE ... RECOVER PARTITIONS",
-          Some("Please use ALTER VIEW instead."))))
+          true)))
   }
 
   test("recover partitions of a table in nested namespaces") {
@@ -69,6 +69,6 @@ class AlterTableRecoverPartitionsParserSuite extends 
AnalysisTest with SharedSpa
         UnresolvedTable(
           Seq("ns1", "ns2", "ns3", "ns4", "ns5", "ns6", "ns7", "ns8", "t"),
           "ALTER TABLE ... RECOVER PARTITIONS",
-          Some("Please use ALTER VIEW instead."))))
+          true)))
   }
 }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala
index 4148798d6cd..16bf0c65217 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenamePartitionParserSuite.scala
@@ -33,7 +33,7 @@ class AlterTableRenamePartitionParserSuite extends 
AnalysisTest with SharedSpark
       UnresolvedTable(
         Seq("a", "b", "c"),
         "ALTER TABLE ... RENAME TO PARTITION",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")),
       UnresolvedPartitionSpec(Map("ds" -> "2018-06-10")))
     comparePlans(parsed, expected)
@@ -49,7 +49,7 @@ class AlterTableRenamePartitionParserSuite extends 
AnalysisTest with SharedSpark
       UnresolvedTable(
         Seq("table_name"),
         "ALTER TABLE ... RENAME TO PARTITION",
-        Some("Please use ALTER VIEW instead.")),
+        true),
       UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
       UnresolvedPartitionSpec(Map("dt" -> "2008-09-09", "country" -> "uk")))
     comparePlans(parsed, expected)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetLocationParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetLocationParserSuite.scala
index 375d4229e96..a44bb8a3729 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetLocationParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetLocationParserSuite.scala
@@ -24,13 +24,11 @@ import org.apache.spark.sql.test.SharedSparkSession
 
 class AlterTableSetLocationParserSuite extends AnalysisTest with 
SharedSparkSession {
 
-  private val HINT = Some("Please use ALTER VIEW instead.")
-
   test("alter table: set location") {
     val sql1 = "ALTER TABLE a.b.c SET LOCATION 'new location'"
     val parsed1 = parsePlan(sql1)
     val expected1 = SetTableLocation(
-      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", 
HINT),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", 
true),
       None,
       "new location")
     comparePlans(parsed1, expected1)
@@ -38,7 +36,7 @@ class AlterTableSetLocationParserSuite extends AnalysisTest 
with SharedSparkSess
     val sql2 = "ALTER TABLE a.b.c PARTITION(ds='2017-06-10') SET LOCATION 'new 
location'"
     val parsed2 = parsePlan(sql2)
     val expected2 = SetTableLocation(
-      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", 
HINT),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ...", 
true),
       Some(Map("ds" -> "2017-06-10")),
       "new location")
     comparePlans(parsed2, expected2)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala
index 1e99801c255..8a3bfd47c6e 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetSerdeParserSuite.scala
@@ -24,8 +24,6 @@ import org.apache.spark.sql.test.SharedSparkSession
 
 class AlterTableSetSerdeParserSuite extends AnalysisTest with 
SharedSparkSession {
 
-  private val HINT = Some("Please use ALTER VIEW instead.")
-
   test("SerDe property values must be set") {
     val sql = "ALTER TABLE table_name SET SERDE 'serde' " +
       "WITH SERDEPROPERTIES('key_without_value', 'key_with_value'='x')"
@@ -43,7 +41,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest with 
SharedSparkSession
     val sql = "ALTER TABLE table_name SET SERDE 'org.apache.class'"
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       Some("org.apache.class"),
       None,
       None)
@@ -58,7 +56,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest with 
SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       Some("org.apache.class"),
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       None)
@@ -73,7 +71,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest with 
SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       None,
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       None)
@@ -89,7 +87,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest with 
SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       Some("org.apache.class"),
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
@@ -104,7 +102,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest 
with SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("table_name"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       None,
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
@@ -120,7 +118,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest 
with SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       Some("org.apache.class"),
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       None)
@@ -136,7 +134,7 @@ class AlterTableSetSerdeParserSuite extends AnalysisTest 
with SharedSparkSession
       """.stripMargin
     val parsed = parsePlan(sql)
     val expected = SetTableSerDeProperties(
-      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", HINT),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET 
[SERDE|SERDEPROPERTIES]", true),
       None,
       Some(Map("columns" -> "foo,bar", "field.delim" -> ",")),
       Some(Map("test" -> "1", "dt" -> "2008-08-08", "country" -> "us")))
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index dea66bb09cf..899d03700b4 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -2064,22 +2064,38 @@ abstract class DDLSuite extends QueryTest with 
DDLSuiteBase {
   test("alter table add columns -- not support temp view") {
     withTempView("tmp_v") {
       sql("CREATE TEMPORARY VIEW tmp_v AS SELECT 1 AS c1, 2 AS c2")
-      val e = intercept[AnalysisException] {
-        sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
-      }
-      assert(e.message.contains(
-        "tmp_v is a temp view. 'ALTER TABLE ... ADD COLUMNS' expects a 
table."))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> "`tmp_v`",
+          "operation" -> "ALTER TABLE ... ADD COLUMNS"),
+        context = ExpectedContext(
+          fragment = "tmp_v",
+          start = 12,
+          stop = 16)
+      )
     }
   }
 
   test("alter table add columns -- not support view") {
     withView("v1") {
       sql("CREATE VIEW v1 AS SELECT 1 AS c1, 2 AS c2")
-      val e = intercept[AnalysisException] {
-        sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
-      }
-      assert(e.message.contains(s"${SESSION_CATALOG_NAME}.default.v1 is a 
view. " +
-        "'ALTER TABLE ... ADD COLUMNS' expects a table."))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
+        },
+        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`v1`",
+          "operation" -> "ALTER TABLE ... ADD COLUMNS"),
+        context = ExpectedContext(
+          fragment = "v1",
+          start = 12,
+          stop = 13)
+      )
     }
   }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/MsckRepairTableParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/MsckRepairTableParserSuite.scala
index f52315a4979..431e4cec232 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/MsckRepairTableParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/MsckRepairTableParserSuite.scala
@@ -26,7 +26,7 @@ class MsckRepairTableParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("MSCK REPAIR TABLE a.b.c"),
       RepairTable(
-        UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE", None),
+        UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE"),
         enableAddPartitions = true,
         enableDropPartitions = false))
   }
@@ -35,7 +35,7 @@ class MsckRepairTableParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("REPAIR TABLE a.b.c"),
       RepairTable(
-        UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE", None),
+        UnresolvedTable(Seq("a", "b", "c"), "MSCK REPAIR TABLE"),
         enableAddPartitions = true,
         enableDropPartitions = false))
   }
@@ -46,8 +46,7 @@ class MsckRepairTableParserSuite extends AnalysisTest {
       RepairTable(
         UnresolvedTable(
           Seq("ns", "tbl"),
-          "MSCK REPAIR TABLE ... ADD PARTITIONS",
-          None),
+          "MSCK REPAIR TABLE ... ADD PARTITIONS"),
         enableAddPartitions = true,
         enableDropPartitions = false))
   }
@@ -58,8 +57,7 @@ class MsckRepairTableParserSuite extends AnalysisTest {
       RepairTable(
         UnresolvedTable(
           Seq("TBL"),
-          "MSCK REPAIR TABLE ... DROP PARTITIONS",
-          None),
+          "MSCK REPAIR TABLE ... DROP PARTITIONS"),
         enableAddPartitions = false,
         enableDropPartitions = true))
   }
@@ -70,8 +68,7 @@ class MsckRepairTableParserSuite extends AnalysisTest {
       RepairTable(
         UnresolvedTable(
           Seq("spark_catalog", "ns", "tbl"),
-          "MSCK REPAIR TABLE ... SYNC PARTITIONS",
-          None),
+          "MSCK REPAIR TABLE ... SYNC PARTITIONS"),
         enableAddPartitions = true,
         enableDropPartitions = true))
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
index d2c363be989..3a5d57c5c78 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
@@ -25,18 +25,18 @@ class ShowPartitionsParserSuite extends AnalysisTest {
   test("SHOW PARTITIONS") {
     val commandName = "SHOW PARTITIONS"
     Seq(
-      "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTable(Seq("t1"), 
commandName, None), None),
+      "SHOW PARTITIONS t1" -> ShowPartitions(UnresolvedTable(Seq("t1"), 
commandName), None),
       "SHOW PARTITIONS db1.t1" -> ShowPartitions(
-        UnresolvedTable(Seq("db1", "t1"), commandName, None), None),
+        UnresolvedTable(Seq("db1", "t1"), commandName), None),
       "SHOW PARTITIONS t1 PARTITION(partcol1='partvalue', 
partcol2='partvalue')" ->
         ShowPartitions(
-          UnresolvedTable(Seq("t1"), commandName, None),
+          UnresolvedTable(Seq("t1"), commandName),
           Some(UnresolvedPartitionSpec(Map("partcol1" -> "partvalue", 
"partcol2" -> "partvalue")))),
       "SHOW PARTITIONS a.b.c" -> ShowPartitions(
-        UnresolvedTable(Seq("a", "b", "c"), commandName, None), None),
+        UnresolvedTable(Seq("a", "b", "c"), commandName), None),
       "SHOW PARTITIONS a.b.c PARTITION(ds='2017-06-10')" ->
         ShowPartitions(
-          UnresolvedTable(Seq("a", "b", "c"), commandName, None),
+          UnresolvedTable(Seq("a", "b", "c"), commandName),
           Some(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"))))
     ).foreach { case (sql, expected) =>
       val parsed = parsePlan(sql)
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
index b78532fc2a0..b903681e341 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
@@ -25,14 +25,14 @@ class TruncateTableParserSuite extends AnalysisTest {
   test("truncate table") {
     comparePlans(
       parsePlan("TRUNCATE TABLE a.b.c"),
-      TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", 
None)))
+      TruncateTable(UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE")))
   }
 
   test("truncate a single part partition") {
     comparePlans(
       parsePlan("TRUNCATE TABLE a.b.c PARTITION(ds='2017-06-10')"),
       TruncatePartition(
-        UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE", None),
+        UnresolvedTable(Seq("a", "b", "c"), "TRUNCATE TABLE"),
         UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"), None)))
   }
 
@@ -40,7 +40,7 @@ class TruncateTableParserSuite extends AnalysisTest {
     comparePlans(
       parsePlan("TRUNCATE TABLE ns.tbl PARTITION(a = 1, B = 'ABC')"),
       TruncatePartition(
-        UnresolvedTable(Seq("ns", "tbl"), "TRUNCATE TABLE", None),
+        UnresolvedTable(Seq("ns", "tbl"), "TRUNCATE TABLE"),
         UnresolvedPartitionSpec(Map("a" -> "1", "B" -> "ABC"), None)))
   }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala
index fbf0e45872a..bbbf63457d0 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableSuiteBase.scala
@@ -177,27 +177,48 @@ trait TruncateTableSuiteBase extends QueryTest with 
DDLCommandTestUtils {
 
       withView("v0") {
         sql(s"CREATE VIEW v0 AS SELECT * FROM $t")
-        val errMsg = intercept[AnalysisException] {
-          sql("TRUNCATE TABLE v0")
-        }.getMessage
-        assert(errMsg.contains("'TRUNCATE TABLE' expects a table"))
+        checkError(
+          exception = intercept[AnalysisException] {
+            sql("TRUNCATE TABLE v0")
+          },
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          parameters = Map(
+            "viewName" -> "`spark_catalog`.`default`.`v0`",
+            "operation" -> "TRUNCATE TABLE"),
+          context = ExpectedContext(
+            fragment = "v0",
+            start = 15,
+            stop = 16)
+        )
       }
 
       withTempView("v1") {
         sql(s"CREATE TEMP VIEW v1 AS SELECT * FROM $t")
-        val errMsg = intercept[AnalysisException] {
-          sql("TRUNCATE TABLE v1")
-        }.getMessage
-        assert(errMsg.contains("'TRUNCATE TABLE' expects a table"))
+        checkError(
+          exception = intercept[AnalysisException] {
+            sql("TRUNCATE TABLE v1")
+          },
+          errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+          parameters = Map(
+            "tempViewName" -> "`v1`",
+            "operation" -> "TRUNCATE TABLE"),
+          context = ExpectedContext(fragment = "v1", start = 15, stop = 16)
+        )
       }
 
       val v2 = s"${spark.sharedState.globalTempViewManager.database}.v2"
       withGlobalTempView("v2") {
         sql(s"CREATE GLOBAL TEMP VIEW v2 AS SELECT * FROM $t")
-        val errMsg = intercept[AnalysisException] {
-          sql(s"TRUNCATE TABLE $v2")
-        }.getMessage
-        assert(errMsg.contains("'TRUNCATE TABLE' expects a table"))
+        checkError(
+          exception = intercept[AnalysisException] {
+            sql(s"TRUNCATE TABLE $v2")
+          },
+          errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+          parameters = Map(
+            "tempViewName" -> "`global_temp`.`v2`",
+            "operation" -> "TRUNCATE TABLE"),
+          context = ExpectedContext(fragment = v2, start = 15, stop = 28)
+        )
       }
     }
   }
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala
index c423bfb9f24..c5dbea63405 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowPartitionsSuite.scala
@@ -53,10 +53,21 @@ trait ShowPartitionsSuiteBase extends 
command.ShowPartitionsSuiteBase {
       val view = "view1"
       withView(view) {
         sql(s"CREATE VIEW $view as select * from $table")
-        val errMsg = intercept[AnalysisException] {
-          sql(s"SHOW PARTITIONS $view")
-        }.getMessage
-        assert(errMsg.contains("'SHOW PARTITIONS' expects a table"))
+        checkError(
+          exception = intercept[AnalysisException] {
+            sql(s"SHOW PARTITIONS $view")
+          },
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          parameters = Map(
+            "viewName" -> s"`spark_catalog`.`default`.`view1`",
+            "operation" -> "SHOW PARTITIONS"
+          ),
+          context = ExpectedContext(
+            fragment = view,
+            start = 16,
+            stop = 20
+          )
+        )
       }
     }
   }
@@ -65,10 +76,21 @@ trait ShowPartitionsSuiteBase extends 
command.ShowPartitionsSuiteBase {
     val viewName = "test_view"
     withTempView(viewName) {
       spark.range(10).createTempView(viewName)
-      val errMsg = intercept[AnalysisException] {
-        sql(s"SHOW PARTITIONS $viewName")
-      }.getMessage
-      assert(errMsg.contains("'SHOW PARTITIONS' expects a table"))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"SHOW PARTITIONS $viewName")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> "`test_view`",
+          "operation" -> "SHOW PARTITIONS"
+        ),
+        context = ExpectedContext(
+          fragment = viewName,
+          start = 16,
+          stop = 24
+        )
+      )
     }
   }
 
@@ -98,10 +120,21 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase 
with CommandSuiteBase
       sql(s"""
         |CREATE TEMPORARY VIEW $viewName (c1 INT, c2 STRING)
         |$defaultUsing""".stripMargin)
-      val errMsg = intercept[AnalysisException] {
-        sql(s"SHOW PARTITIONS $viewName")
-      }.getMessage
-      assert(errMsg.contains("'SHOW PARTITIONS' expects a table"))
+      checkError(
+        exception = intercept[AnalysisException] {
+          sql(s"SHOW PARTITIONS $viewName")
+        },
+        errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+        parameters = Map(
+          "tempViewName" -> "`test_view`",
+          "operation" -> "SHOW PARTITIONS"
+        ),
+        context = ExpectedContext(
+          fragment = viewName,
+          start = 16,
+          stop = 24
+        )
+      )
     }
   }
 
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
index c6bf220e45d..f1eb3bc19af 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
@@ -709,10 +709,15 @@ class CatalogSuite extends SharedSparkSession with 
AnalysisTest with BeforeAndAf
 
   test("SPARK-34301: recover partitions of views is not supported") {
     createTempTable("my_temp_table")
-    val errMsg = intercept[AnalysisException] {
-      spark.catalog.recoverPartitions("my_temp_table")
-    }.getMessage
-    assert(errMsg.contains("my_temp_table is a temp view. 
'recoverPartitions()' expects a table"))
+    checkError(
+      exception = intercept[AnalysisException] {
+        spark.catalog.recoverPartitions("my_temp_table")
+      },
+      errorClass = "UNSUPPORTED_TEMP_VIEW_OPERATION.WITH_SUGGESTION",
+      parameters = Map(
+        "tempViewName" -> "`my_temp_table`",
+        "operation" -> "recoverPartitions()")
+    )
   }
 
   test("qualified name with catalog - create managed table") {
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index 006caa02d55..1c46b558708 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -896,11 +896,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1015",
+          errorClass = "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "identifier" -> s"default.$tabName",
-            "cmd" -> "ALTER VIEW ... SET TBLPROPERTIES",
-            "hintStr" -> " Please use ALTER TABLE instead."),
+            "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`",
+            "operation" -> "ALTER VIEW ... SET TBLPROPERTIES"),
           context = ExpectedContext(fragment = tabName, start = 11, stop = 14)
         )
 
@@ -908,12 +907,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... SET TBLPROPERTIES",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... SET TBLPROPERTIES"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -921,11 +918,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1015",
+          errorClass = "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "identifier" -> s"default.$tabName",
-            "cmd" -> "ALTER VIEW ... UNSET TBLPROPERTIES",
-            "hintStr" -> " Please use ALTER TABLE instead."),
+            "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`",
+            "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES"),
           context = ExpectedContext(fragment = tabName, start = 11, stop = 14)
         )
 
@@ -933,12 +929,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... UNSET TBLPROPERTIES",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -946,12 +940,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... SET LOCATION ...",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... SET LOCATION ..."),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -959,12 +951,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET SERDE 'whatever'")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -972,12 +962,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -985,12 +973,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET 
SERDEPROPERTIES ('x' = 'y')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -998,12 +984,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName RECOVER PARTITIONS")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... RECOVER PARTITIONS",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... RECOVER PARTITIONS"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -1011,12 +995,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO 
PARTITION (a='100')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... RENAME TO PARTITION",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... RENAME TO PARTITION"),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -1024,12 +1006,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', 
b='8')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... ADD PARTITION ...",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... ADD PARTITION ..."),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 
@@ -1037,12 +1017,10 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')")
           },
-          errorClass = "_LEGACY_ERROR_TEMP_1013",
+          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
           parameters = Map(
-            "nameParts" -> s"$SESSION_CATALOG_NAME.default.$oldViewName",
-            "viewStr" -> "view",
-            "cmd" -> "ALTER TABLE ... DROP PARTITION ...",
-            "hintStr" -> " Please use ALTER VIEW instead."),
+            "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
+            "operation" -> "ALTER TABLE ... DROP PARTITION ..."),
           context = ExpectedContext(fragment = oldViewName, start = 12, stop = 
16)
         )
 


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

Reply via email to