You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2023/09/22 01:25:04 UTC

[spark] branch master updated: [SPARK-45163][SQL] Merge UNSUPPORTED_VIEW_OPERATION & UNSUPPORTED_TABLE_OPERATION & fix some issue

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

wenchen 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 39d6fdabb48 [SPARK-45163][SQL] Merge UNSUPPORTED_VIEW_OPERATION & UNSUPPORTED_TABLE_OPERATION & fix some issue
39d6fdabb48 is described below

commit 39d6fdabb48c700ab5d9fe33437341f5dbf3d1d7
Author: panbingkun <pb...@gmail.com>
AuthorDate: Fri Sep 22 09:24:48 2023 +0800

    [SPARK-45163][SQL] Merge UNSUPPORTED_VIEW_OPERATION & UNSUPPORTED_TABLE_OPERATION & fix some issue
    
    ### What changes were proposed in this pull request?
    A.The pr aims to
    - Merge `UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION` into `EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW` (new added)
    - Merge `UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION ` into `EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE`
    - Merge `UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION` into       `EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE`
    - Merge `UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION` into `EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE`
    - Merge `UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION` into `EXPECT_VIEW_NOT_TABLE.USE_ALTER_VIEW`
    - Add `EXPECT_PERMANENT_VIEW_NOT_TEMP`
    - Fix some naming issues based on the suggestions of [PR](https://github.com/apache/spark/pull/42824) reviewers.
    
    B.The pr is also follow up https://github.com/apache/spark/pull/42824.
    
    ### Why are the changes needed?
    - Better code readability.
    - Fix some error message prompt.
    - Fix a type: "ALTER COLUMN ... FIRST | ALTER" -> "ALTER COLUMN ... FIRST | AFTER"
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    Pass GA.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #42917 from panbingkun/SPARK-45085_FOLLOWUP.
    
    Authored-by: panbingkun <pb...@gmail.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 R/pkg/tests/fulltests/test_sparkSQL.R              |  2 +-
 .../src/main/resources/error/error-classes.json    | 73 ++++++++++++----------
 ...onditions-expect-table-not-view-error-class.md} | 10 +--
 ...onditions-expect-view-not-table-error-class.md} |  8 +--
 docs/sql-error-conditions.md                       | 38 ++++++-----
 .../spark/sql/catalyst/analysis/Analyzer.scala     | 10 +--
 .../sql/catalyst/analysis/v2ResolutionPlans.scala  |  2 +-
 .../spark/sql/catalyst/parser/AstBuilder.scala     | 27 ++++----
 .../spark/sql/errors/QueryCompilationErrors.scala  | 53 +++++++++++-----
 .../catalyst/analysis/ResolveSessionCatalog.scala  |  2 +-
 .../apache/spark/sql/execution/command/views.scala |  2 +-
 .../analyzer-results/change-column.sql.out         |  8 +--
 .../sql-tests/results/change-column.sql.out        |  8 +--
 .../spark/sql/connector/DataSourceV2SQLSuite.scala |  2 +-
 .../apache/spark/sql/execution/SQLViewSuite.scala  | 45 +++++++------
 .../spark/sql/execution/SQLViewTestSuite.scala     |  2 +-
 .../AlterTableAddPartitionParserSuite.scala        |  6 +-
 .../AlterTableDropPartitionParserSuite.scala       | 12 ++--
 .../AlterTableRecoverPartitionsParserSuite.scala   | 12 ++--
 .../AlterTableRenamePartitionParserSuite.scala     |  6 +-
 .../command/AlterTableSetLocationParserSuite.scala |  4 +-
 .../spark/sql/execution/command/DDLSuite.scala     |  6 +-
 .../execution/command/TruncateTableSuiteBase.scala |  6 +-
 .../execution/command/v1/ShowPartitionsSuite.scala |  6 +-
 .../apache/spark/sql/internal/CatalogSuite.scala   |  2 +-
 .../spark/sql/hive/execution/HiveDDLSuite.scala    | 24 +++----
 26 files changed, 196 insertions(+), 180 deletions(-)

diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R
index 4d3c2349f16..f2bef7a0044 100644
--- a/R/pkg/tests/fulltests/test_sparkSQL.R
+++ b/R/pkg/tests/fulltests/test_sparkSQL.R
@@ -4199,7 +4199,7 @@ test_that("catalog APIs, listTables, getTable, listColumns, listFunctions, funct
 
   # recoverPartitions does not work with temporary view
   expect_error(recoverPartitions("cars"),
-               "[UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION]*`cars`*")
+               "[EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE]*`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 8942d3755e9..9bcbcbc1962 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -877,6 +877,45 @@
       "Exceeds char/varchar type length limitation: <limit>."
     ]
   },
+  "EXPECT_PERMANENT_VIEW_NOT_TEMP" : {
+    "message" : [
+      "'<operation>' expects a permanent view but <viewName> is a temp view."
+    ]
+  },
+  "EXPECT_TABLE_NOT_VIEW" : {
+    "message" : [
+      "'<operation>' expects a table but <viewName> is a view."
+    ],
+    "subClass" : {
+      "NO_ALTERNATIVE" : {
+        "message" : [
+          ""
+        ]
+      },
+      "USE_ALTER_VIEW" : {
+        "message" : [
+          "Please use ALTER VIEW instead."
+        ]
+      }
+    }
+  },
+  "EXPECT_VIEW_NOT_TABLE" : {
+    "message" : [
+      "The table <tableName> does not support <operation>."
+    ],
+    "subClass" : {
+      "NO_ALTERNATIVE" : {
+        "message" : [
+          ""
+        ]
+      },
+      "USE_ALTER_TABLE" : {
+        "message" : [
+          "Please use ALTER TABLE instead."
+        ]
+      }
+    }
+  },
   "EXPRESSION_TYPE_IS_NOT_ORDERABLE" : {
     "message" : [
       "Column expression <expr> cannot be sorted because its type <exprType> is not orderable."
@@ -3444,46 +3483,12 @@
     },
     "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_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:",
diff --git a/docs/sql-error-conditions-unsupported-view-operation-error-class.md b/docs/sql-error-conditions-expect-table-not-view-error-class.md
similarity index 82%
rename from docs/sql-error-conditions-unsupported-view-operation-error-class.md
rename to docs/sql-error-conditions-expect-table-not-view-error-class.md
index 88f3adc3fdb..9be7fad6cc7 100644
--- a/docs/sql-error-conditions-unsupported-view-operation-error-class.md
+++ b/docs/sql-error-conditions-expect-table-not-view-error-class.md
@@ -1,7 +1,7 @@
 ---
 layout: global
-title: UNSUPPORTED_VIEW_OPERATION error class
-displayTitle: UNSUPPORTED_VIEW_OPERATION error class
+title: EXPECT_TABLE_NOT_VIEW error class
+displayTitle: EXPECT_TABLE_NOT_VIEW error class
 license: |
   Licensed to the Apache Software Foundation (ASF) under one or more
   contributor license agreements.  See the NOTICE file distributed with
@@ -21,15 +21,15 @@ license: |
 
 SQLSTATE: none assigned
 
-The view `<viewName>` does not support `<operation>`.
+'`<operation>`' expects a table but `<viewName>` is a view.
 
 This error class has the following derived error classes:
 
-## WITHOUT_SUGGESTION
+## NO_ALTERNATIVE
 
 
 
-## WITH_SUGGESTION
+## USE_ALTER_VIEW
 
 Please use ALTER VIEW instead.
 
diff --git a/docs/sql-error-conditions-unsupported-table-operation-error-class.md b/docs/sql-error-conditions-expect-view-not-table-error-class.md
similarity index 87%
rename from docs/sql-error-conditions-unsupported-table-operation-error-class.md
rename to docs/sql-error-conditions-expect-view-not-table-error-class.md
index 5e4c07ccffd..d4ed0d9457c 100644
--- a/docs/sql-error-conditions-unsupported-table-operation-error-class.md
+++ b/docs/sql-error-conditions-expect-view-not-table-error-class.md
@@ -1,7 +1,7 @@
 ---
 layout: global
-title: UNSUPPORTED_TABLE_OPERATION error class
-displayTitle: UNSUPPORTED_TABLE_OPERATION error class
+title: EXPECT_VIEW_NOT_TABLE error class
+displayTitle: EXPECT_VIEW_NOT_TABLE error class
 license: |
   Licensed to the Apache Software Foundation (ASF) under one or more
   contributor license agreements.  See the NOTICE file distributed with
@@ -25,11 +25,11 @@ The table `<tableName>` does not support `<operation>`.
 
 This error class has the following derived error classes:
 
-## WITHOUT_SUGGESTION
+## NO_ALTERNATIVE
 
 
 
-## WITH_SUGGESTION
+## USE_ALTER_TABLE
 
 Please use ALTER TABLE instead.
 
diff --git a/docs/sql-error-conditions.md b/docs/sql-error-conditions.md
index f6f94efc2b0..38cfc28ba09 100644
--- a/docs/sql-error-conditions.md
+++ b/docs/sql-error-conditions.md
@@ -523,6 +523,28 @@ SQLSTATE: none assigned
 
 Exceeds char/varchar type length limitation: `<limit>`.
 
+### EXPECT_PERMANENT_VIEW_NOT_TEMP
+
+SQLSTATE: none assigned
+
+'`<operation>`' expects a permanent view but `<viewName>` is a temp view.
+
+### [EXPECT_TABLE_NOT_VIEW](sql-error-conditions-expect-table-not-view-error-class.html)
+
+SQLSTATE: none assigned
+
+'`<operation>`' expects a table but `<viewName>` is a view.
+
+For more details see [EXPECT_TABLE_NOT_VIEW](sql-error-conditions-expect-table-not-view-error-class.html)
+
+### [EXPECT_VIEW_NOT_TABLE](sql-error-conditions-expect-view-not-table-error-class.html)
+
+SQLSTATE: none assigned
+
+The table `<tableName>` does not support `<operation>`.
+
+For more details see [EXPECT_VIEW_NOT_TABLE](sql-error-conditions-expect-view-not-table-error-class.html)
+
 ### EXPRESSION_TYPE_IS_NOT_ORDERABLE
 
 SQLSTATE: none assigned
@@ -2106,28 +2128,12 @@ 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_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
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 6491a4eea95..cff29de858e 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
@@ -1128,10 +1128,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
         lookupTableOrView(identifier).map {
           case v: ResolvedPersistentView =>
             val nameParts = v.catalog.name() +: v.identifier.asMultipartIdentifier
-            throw QueryCompilationErrors.unsupportedViewOperationError(
+            throw QueryCompilationErrors.expectTableNotViewError(
               nameParts, cmd, suggestAlternative, u)
           case _: ResolvedTempView =>
-            throw QueryCompilationErrors.unsupportedViewOperationError(
+            throw QueryCompilationErrors.expectTableNotViewError(
               identifier, cmd, suggestAlternative, u)
           case table => table
         }.getOrElse(u)
@@ -1139,7 +1139,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
       case u @ UnresolvedView(identifier, cmd, allowTemp, suggestAlternative) =>
         lookupTableOrView(identifier, viewOnly = true).map {
           case _: ResolvedTempView if !allowTemp =>
-            throw QueryCompilationErrors.unsupportedViewOperationError(identifier, cmd, false, u)
+            throw QueryCompilationErrors.expectPermanentViewNotTempViewError(
+              identifier, cmd, u)
           case t: ResolvedTable =>
             val nameParts = t.catalog.name() +: t.identifier.asMultipartIdentifier
             throw QueryCompilationErrors.expectViewNotTableError(
@@ -1150,7 +1151,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
       case u @ UnresolvedTableOrView(identifier, cmd, allowTempView) =>
         lookupTableOrView(identifier).map {
           case _: ResolvedTempView if !allowTempView =>
-            throw QueryCompilationErrors.unsupportedViewOperationError(identifier, cmd, false, u)
+            throw QueryCompilationErrors.expectPermanentViewNotTempViewError(
+              identifier, cmd, 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 0306385cc46..6634ce72d7b 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
@@ -54,7 +54,7 @@ case class UnresolvedView(
     multipartIdentifier: Seq[String],
     commandName: String,
     allowTemp: Boolean,
-    suggestAlternative: Boolean) extends UnresolvedLeafNode
+    suggestAlternative: Boolean = false) 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 f8a9ea268d2..10628bb306a 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,
-      hint: Boolean = false): LogicalPlan = withOrigin(ctx) {
-    withIdentClause(ctx, UnresolvedTable(_, commandName, hint))
+      suggestAlternative: Boolean = false): LogicalPlan = withOrigin(ctx) {
+    withIdentClause(ctx, UnresolvedTable(_, commandName, suggestAlternative))
   }
 
   /**
@@ -2823,8 +2823,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
       ctx: IdentifierReferenceContext,
       commandName: String,
       allowTemp: Boolean = true,
-      hint: Boolean = false): LogicalPlan = withOrigin(ctx) {
-    withIdentClause(ctx, UnresolvedView(_, commandName, allowTemp, hint))
+      suggestAlternative: Boolean = false): LogicalPlan = withOrigin(ctx) {
+    withIdentClause(ctx, UnresolvedView(_, commandName, allowTemp, suggestAlternative))
   }
 
   /**
@@ -4359,7 +4359,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
           ctx.identifierReference,
           commandName = "ALTER VIEW ... SET TBLPROPERTIES",
           allowTemp = false,
-          hint = true),
+          suggestAlternative = true),
         cleanedTableProperties)
     } else {
       SetTableProperties(
@@ -4392,7 +4392,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
           ctx.identifierReference,
           commandName = "ALTER VIEW ... UNSET TBLPROPERTIES",
           allowTemp = false,
-          hint = true),
+          suggestAlternative = true),
         cleanedProperties,
         ifExists)
     } else {
@@ -4418,8 +4418,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     SetTableLocation(
       createUnresolvedTable(
         ctx.identifierReference,
-        "ALTER TABLE ... SET LOCATION ...",
-        true),
+        "ALTER TABLE ... SET LOCATION ..."),
       Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec),
       visitLocationSpec(ctx.locationSpec))
   }
@@ -4715,8 +4714,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     RecoverPartitions(
       createUnresolvedTable(
         ctx.identifierReference,
-        "ALTER TABLE ... RECOVER PARTITIONS",
-        true))
+        "ALTER TABLE ... RECOVER PARTITIONS"))
   }
 
   /**
@@ -4745,8 +4743,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     AddPartitions(
       createUnresolvedTable(
         ctx.identifierReference,
-        "ALTER TABLE ... ADD PARTITION ...",
-        true),
+        "ALTER TABLE ... ADD PARTITION ..."),
       specsAndLocs.toSeq,
       ctx.EXISTS != null)
   }
@@ -4764,8 +4761,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     RenamePartitions(
       createUnresolvedTable(
         ctx.identifierReference,
-        "ALTER TABLE ... RENAME TO PARTITION",
-        true),
+        "ALTER TABLE ... RENAME TO PARTITION"),
       UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.from)),
       UnresolvedPartitionSpec(visitNonOptionalPartitionSpec(ctx.to)))
   }
@@ -4793,8 +4789,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     DropPartitions(
       createUnresolvedTable(
         ctx.identifierReference,
-        "ALTER TABLE ... DROP PARTITION ...",
-        true),
+        "ALTER TABLE ... DROP PARTITION ..."),
       partSpecs.toSeq,
       ifExists = ctx.EXISTS != null,
       purge = ctx.PURGE != null)
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 7b83844ff33..3536626d239 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
@@ -460,7 +460,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
 
   def insertIntoViewNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+      errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
       messageParameters = Map(
         "viewName" -> toSQLId(identifier.nameParts),
         "operation" -> "INSERT"),
@@ -481,16 +481,16 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       origin = t.origin)
   }
 
-  def unsupportedViewOperationError(
+  def expectTableNotViewError(
       nameParts: Seq[String],
       cmd: String,
       suggestAlternative: Boolean,
       t: TreeNode[_]): Throwable = {
     new AnalysisException(
       errorClass = if (suggestAlternative) {
-        "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION"
+        "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW"
       } else {
-        "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION"
+        "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE"
       },
       messageParameters = Map(
         "viewName" -> toSQLId(nameParts),
@@ -501,13 +501,13 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
   def expectViewNotTableError(
       nameParts: Seq[String],
       cmd: String,
-      hint: Boolean,
+      suggestAlternative: Boolean,
       t: TreeNode[_]): Throwable = {
     new AnalysisException(
-      errorClass = if (hint) {
-        "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION"
+      errorClass = if (suggestAlternative) {
+        "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE"
       } else {
-        "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION"
+        "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE"
       },
       messageParameters = Map(
         "tableName" -> toSQLId(nameParts),
@@ -515,6 +515,18 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
       origin = t.origin)
   }
 
+  def expectPermanentViewNotTempViewError(
+      nameParts: Seq[String],
+      cmd: String,
+      t: TreeNode[_]): Throwable = {
+    new AnalysisException(
+      errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
+      messageParameters = Map(
+        "viewName" -> toSQLId(nameParts),
+        "operation" -> cmd),
+      origin = t.origin)
+  }
+
   def expectPersistentFuncError(
       name: String, cmd: String, mismatchHint: Option[String], t: TreeNode[_]): Throwable = {
     val hintStr = mismatchHint.map(" " + _).getOrElse("")
@@ -2854,15 +2866,24 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
         "dataColumns" -> query.output.map(c => toSQLId(c.name)).mkString(", ")))
   }
 
-  def tableIsNotViewError(name: TableIdentifier, replace: Boolean): Throwable = {
-    val operation = if (replace) "CREATE OR REPLACE VIEW" else "CREATE VIEW"
-    new AnalysisException(
-      errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
-      messageParameters = Map(
-        "tableName" -> toSQLId(name.nameParts),
-        "operation" -> operation
+  def unsupportedCreateOrReplaceViewOnTableError(
+      name: TableIdentifier, replace: Boolean): Throwable = {
+    if (replace) {
+      new AnalysisException(
+        errorClass = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE",
+        messageParameters = Map(
+          "tableName" -> toSQLId(name.nameParts),
+          "operation" -> "CREATE OR REPLACE VIEW"
+        )
       )
-    )
+    } else {
+      new AnalysisException(
+        errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS",
+        messageParameters = Map(
+          "relationName" -> toSQLId(name.nameParts)
+        )
+      )
+    }
   }
 
   def viewAlreadyExistsError(name: TableIdentifier): Throwable = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
index 23a149fc03a..033fe5b0d1f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala
@@ -76,7 +76,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
       }
       if (a.position.isDefined) {
         throw QueryCompilationErrors.unsupportedTableOperationError(
-          catalog, ident, "ALTER COLUMN ... FIRST | ALTER")
+          catalog, ident, "ALTER COLUMN ... FIRST | AFTER")
       }
       val builder = new MetadataBuilder
       // Add comment to metadata
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 b27bd5bf909..88b7826bd91 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, replace)
+        throw QueryCompilationErrors.unsupportedCreateOrReplaceViewOnTableError(name, replace)
       } else if (replace) {
         // Detect cyclic view reference on CREATE OR REPLACE VIEW.
         val viewIdent = tableMetadata.identifier
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 b13c603e71c..9fdf09c4c65 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
@@ -86,7 +86,7 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
   "sqlState" : "0A000",
   "messageParameters" : {
-    "operation" : "ALTER COLUMN ... FIRST | ALTER",
+    "operation" : "ALTER COLUMN ... FIRST | AFTER",
     "tableName" : "`spark_catalog`.`default`.`test_change`"
   }
 }
@@ -100,7 +100,7 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
   "sqlState" : "0A000",
   "messageParameters" : {
-    "operation" : "ALTER COLUMN ... FIRST | ALTER",
+    "operation" : "ALTER COLUMN ... FIRST | AFTER",
     "tableName" : "`spark_catalog`.`default`.`test_change`"
   }
 }
@@ -206,7 +206,7 @@ ALTER TABLE temp_view CHANGE a TYPE INT
 -- !query analysis
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+  "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
   "messageParameters" : {
     "operation" : "ALTER TABLE ... CHANGE COLUMN",
     "viewName" : "`temp_view`"
@@ -234,7 +234,7 @@ ALTER TABLE global_temp.global_temp_view CHANGE a TYPE INT
 -- !query analysis
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+  "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
   "messageParameters" : {
     "operation" : "ALTER TABLE ... CHANGE COLUMN",
     "viewName" : "`global_temp`.`global_temp_view`"
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 83d0ef57001..0b7a6e88f51 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
@@ -112,7 +112,7 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
   "sqlState" : "0A000",
   "messageParameters" : {
-    "operation" : "ALTER COLUMN ... FIRST | ALTER",
+    "operation" : "ALTER COLUMN ... FIRST | AFTER",
     "tableName" : "`spark_catalog`.`default`.`test_change`"
   }
 }
@@ -128,7 +128,7 @@ org.apache.spark.sql.AnalysisException
   "errorClass" : "UNSUPPORTED_FEATURE.TABLE_OPERATION",
   "sqlState" : "0A000",
   "messageParameters" : {
-    "operation" : "ALTER COLUMN ... FIRST | ALTER",
+    "operation" : "ALTER COLUMN ... FIRST | AFTER",
     "tableName" : "`spark_catalog`.`default`.`test_change`"
   }
 }
@@ -270,7 +270,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+  "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
   "messageParameters" : {
     "operation" : "ALTER TABLE ... CHANGE COLUMN",
     "viewName" : "`temp_view`"
@@ -300,7 +300,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+  "errorClass" : "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
   "messageParameters" : {
     "operation" : "ALTER TABLE ... CHANGE COLUMN",
     "viewName" : "`global_temp`.`global_temp_view`"
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 c8ad7ec02a9..8e0580bf644 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,7 +2601,7 @@ 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 = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> "`global_temp`.`v`",
           "operation" -> "COMMENT ON TABLE"),
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 ff483aa4da6..75e5d4d452e 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,7 +115,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql("CREATE OR REPLACE VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE",
         parameters = Map(
           "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
           "operation" -> "CREATE OR REPLACE VIEW")
@@ -124,16 +124,15 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql("CREATE VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "TABLE_OR_VIEW_ALREADY_EXISTS",
         parameters = Map(
-          "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
-          "operation" -> "CREATE VIEW")
+          "relationName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`")
       )
       checkError(
         exception = intercept[AnalysisException] {
           sql("ALTER VIEW tab1 AS SELECT * FROM jt")
         },
-        errorClass = "UNSUPPORTED_TABLE_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_VIEW_NOT_TABLE.NO_ALTERNATIVE",
         parameters = Map(
           "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`tab1`",
           "operation" -> "ALTER VIEW ... AS"
@@ -162,7 +161,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER VIEW $viewName SET TBLPROPERTIES ('p' = 'an')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER VIEW ... SET TBLPROPERTIES"
@@ -177,7 +176,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES"
@@ -199,7 +198,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName SET SERDE 'whatever'")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
@@ -210,7 +209,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName PARTITION (a=1, b=2) SET SERDE 'whatever'")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
@@ -221,7 +220,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName SET SERDEPROPERTIES ('p' = 'an')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"
@@ -232,7 +231,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName PARTITION (a='4') RENAME TO PARTITION (a='5')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... RENAME TO PARTITION"
@@ -243,7 +242,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName RECOVER PARTITIONS")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... RECOVER PARTITIONS"
@@ -254,7 +253,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName SET LOCATION '/path/to/your/lovely/heart'")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET LOCATION ..."
@@ -265,7 +264,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName PARTITION (a='4') SET LOCATION '/path/to/home'")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET LOCATION ..."
@@ -276,7 +275,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName ADD IF NOT EXISTS PARTITION (a='4', b='8')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... ADD PARTITION ..."
@@ -287,7 +286,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName DROP PARTITION (a='4', b='8')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... DROP PARTITION ..."
@@ -298,7 +297,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName SET TBLPROPERTIES ('p' = 'an')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... SET TBLPROPERTIES"
@@ -309,7 +308,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ALTER TABLE $viewName UNSET TBLPROPERTIES ('p')")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES"
@@ -339,7 +338,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(sqlText)
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "LOAD DATA"
@@ -354,7 +353,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"SHOW CREATE TABLE $viewName")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "SHOW CREATE TABLE"
@@ -369,7 +368,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"ANALYZE TABLE $viewName COMPUTE STATISTICS")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
         parameters = Map(
           "viewName" -> s"`$viewName`",
           "operation" -> "ANALYZE TABLE"
@@ -406,7 +405,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(s"INSERT INTO TABLE $viewName SELECT 1")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`",
           "operation" -> "INSERT"
@@ -421,7 +420,7 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils {
         exception = intercept[AnalysisException] {
           sql(sqlText)
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`testview`",
           "operation" -> "LOAD DATA"),
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 c62227626b1..73d0bd19bf6 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
@@ -488,7 +488,7 @@ abstract class TempViewTestSuite extends SQLViewTestSuite {
         exception = intercept[AnalysisException] {
           sql(s"SHOW CREATE TABLE ${formattedViewName(viewName)}")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_PERMANENT_VIEW_NOT_TEMP",
         parameters = Map(
           "viewName" -> toSQLId(tableIdentifier(viewName).nameParts),
           "operation" -> "SHOW CREATE TABLE"),
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 90b1587f397..73430e8fe5e 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
@@ -32,8 +32,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes
     val expected = AddPartitions(
       UnresolvedTable(
         Seq("a", "b", "c"),
-        "ALTER TABLE ... ADD PARTITION ...",
-        true),
+        "ALTER TABLE ... ADD PARTITION ..."),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us"), Some("location1")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"), None)),
@@ -47,8 +46,7 @@ class AlterTableAddPartitionParserSuite extends AnalysisTest with SharedSparkSes
     val expected = AddPartitions(
       UnresolvedTable(
         Seq("a", "b", "c"),
-        "ALTER TABLE ... ADD PARTITION ...",
-        true),
+        "ALTER TABLE ... ADD PARTITION ..."),
       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 7d6b5730c9c..605d736673c 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
@@ -32,8 +32,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe
     val expected = DropPartitions(
       UnresolvedTable(
         Seq("table_name"),
-        "ALTER TABLE ... DROP PARTITION ...",
-        true),
+        "ALTER TABLE ... DROP PARTITION ..."),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))),
@@ -52,8 +51,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe
     val expected = DropPartitions(
       UnresolvedTable(
         Seq("table_name"),
-        "ALTER TABLE ... DROP PARTITION ...",
-        true),
+        "ALTER TABLE ... DROP PARTITION ..."),
       Seq(
         UnresolvedPartitionSpec(Map("dt" -> "2008-08-08", "country" -> "us")),
         UnresolvedPartitionSpec(Map("dt" -> "2009-09-09", "country" -> "uk"))),
@@ -67,8 +65,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe
     val expected = DropPartitions(
       UnresolvedTable(
         Seq("a", "b", "c"),
-        "ALTER TABLE ... DROP PARTITION ...",
-        true),
+        "ALTER TABLE ... DROP PARTITION ..."),
       Seq(UnresolvedPartitionSpec(Map("ds" -> "2017-06-10"))),
       ifExists = true,
       purge = false)
@@ -81,8 +78,7 @@ class AlterTableDropPartitionParserSuite extends AnalysisTest with SharedSparkSe
     val expected = DropPartitions(
       UnresolvedTable(
         Seq("table_name"),
-        "ALTER TABLE ... DROP PARTITION ...",
-        true),
+        "ALTER TABLE ... DROP PARTITION ..."),
       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 7c82b1f81ab..936b1a3dfdb 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
@@ -38,8 +38,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa
       RecoverPartitions(
         UnresolvedTable(
           Seq("tbl"),
-          "ALTER TABLE ... RECOVER PARTITIONS",
-          true)))
+          "ALTER TABLE ... RECOVER PARTITIONS")))
   }
 
   test("recover partitions of a table in a database") {
@@ -48,8 +47,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa
       RecoverPartitions(
         UnresolvedTable(
           Seq("db", "tbl"),
-          "ALTER TABLE ... RECOVER PARTITIONS",
-          true)))
+          "ALTER TABLE ... RECOVER PARTITIONS")))
   }
 
   test("recover partitions of a table spark_catalog") {
@@ -58,8 +56,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa
       RecoverPartitions(
         UnresolvedTable(
           Seq("spark_catalog", "db", "TBL"),
-          "ALTER TABLE ... RECOVER PARTITIONS",
-          true)))
+          "ALTER TABLE ... RECOVER PARTITIONS")))
   }
 
   test("recover partitions of a table in nested namespaces") {
@@ -68,7 +65,6 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa
       RecoverPartitions(
         UnresolvedTable(
           Seq("ns1", "ns2", "ns3", "ns4", "ns5", "ns6", "ns7", "ns8", "t"),
-          "ALTER TABLE ... RECOVER PARTITIONS",
-          true)))
+          "ALTER TABLE ... RECOVER PARTITIONS")))
   }
 }
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 16bf0c65217..848d17bf15f 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
@@ -32,8 +32,7 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark
     val expected = RenamePartitions(
       UnresolvedTable(
         Seq("a", "b", "c"),
-        "ALTER TABLE ... RENAME TO PARTITION",
-        true),
+        "ALTER TABLE ... RENAME TO PARTITION"),
       UnresolvedPartitionSpec(Map("ds" -> "2017-06-10")),
       UnresolvedPartitionSpec(Map("ds" -> "2018-06-10")))
     comparePlans(parsed, expected)
@@ -48,8 +47,7 @@ class AlterTableRenamePartitionParserSuite extends AnalysisTest with SharedSpark
     val expected = RenamePartitions(
       UnresolvedTable(
         Seq("table_name"),
-        "ALTER TABLE ... RENAME TO PARTITION",
-        true),
+        "ALTER TABLE ... RENAME TO PARTITION"),
       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 a44bb8a3729..3f50d6316d4 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
@@ -28,7 +28,7 @@ class AlterTableSetLocationParserSuite extends AnalysisTest with SharedSparkSess
     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 ...", true),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."),
       None,
       "new location")
     comparePlans(parsed1, expected1)
@@ -36,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 ...", true),
+      UnresolvedTable(Seq("a", "b", "c"), "ALTER TABLE ... SET LOCATION ..."),
       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/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index f550ac6f08e..f0a5f14bc40 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
@@ -211,7 +211,7 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSparkSession {
         errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION",
         sqlState = "0A000",
         parameters = Map("tableName" -> "`spark_catalog`.`default`.`t`",
-          "operation" -> "ALTER COLUMN ... FIRST | ALTER"))
+          "operation" -> "ALTER COLUMN ... FIRST | AFTER"))
     }
   }
 
@@ -2068,7 +2068,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase {
         exception = intercept[AnalysisException] {
           sql("ALTER TABLE tmp_v ADD COLUMNS (c3 INT)")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> "`tmp_v`",
           "operation" -> "ALTER TABLE ... ADD COLUMNS"),
@@ -2087,7 +2087,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase {
         exception = intercept[AnalysisException] {
           sql("ALTER TABLE v1 ADD COLUMNS (c3 INT)")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`v1`",
           "operation" -> "ALTER TABLE ... ADD COLUMNS"),
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 431751c9c29..facbfa3dedf 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
@@ -181,7 +181,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils {
           exception = intercept[AnalysisException] {
             sql("TRUNCATE TABLE v0")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> "`spark_catalog`.`default`.`v0`",
             "operation" -> "TRUNCATE TABLE"),
@@ -198,7 +198,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils {
           exception = intercept[AnalysisException] {
             sql("TRUNCATE TABLE v1")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> "`v1`",
             "operation" -> "TRUNCATE TABLE"),
@@ -213,7 +213,7 @@ trait TruncateTableSuiteBase extends QueryTest with DDLCommandTestUtils {
           exception = intercept[AnalysisException] {
             sql(s"TRUNCATE TABLE $v2")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> "`global_temp`.`v2`",
             "operation" -> "TRUNCATE TABLE"),
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 e282e20bdea..9863942c6ea 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
@@ -57,7 +57,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase {
           exception = intercept[AnalysisException] {
             sql(s"SHOW PARTITIONS $view")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`spark_catalog`.`default`.`view1`",
             "operation" -> "SHOW PARTITIONS"
@@ -80,7 +80,7 @@ trait ShowPartitionsSuiteBase extends command.ShowPartitionsSuiteBase {
         exception = intercept[AnalysisException] {
           sql(s"SHOW PARTITIONS $viewName")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> "`test_view`",
           "operation" -> "SHOW PARTITIONS"
@@ -124,7 +124,7 @@ class ShowPartitionsSuite extends ShowPartitionsSuiteBase with CommandSuiteBase
         exception = intercept[AnalysisException] {
           sql(s"SHOW PARTITIONS $viewName")
         },
-        errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+        errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
         parameters = Map(
           "viewName" -> "`test_view`",
           "operation" -> "SHOW PARTITIONS"
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 518302f811e..01e23d56819 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
@@ -713,7 +713,7 @@ class CatalogSuite extends SharedSparkSession with AnalysisTest with BeforeAndAf
       exception = intercept[AnalysisException] {
         spark.catalog.recoverPartitions("my_temp_table")
       },
-      errorClass = "UNSUPPORTED_VIEW_OPERATION.WITHOUT_SUGGESTION",
+      errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
       parameters = Map(
         "viewName" -> "`my_temp_table`",
         "operation" -> "recoverPartitions()")
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 99250ed6a91..b6971204b1c 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
@@ -876,7 +876,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')")
           },
-          errorClass = "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE",
           parameters = Map(
             "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`",
             "operation" -> "ALTER VIEW ... SET TBLPROPERTIES"),
@@ -887,7 +887,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... SET TBLPROPERTIES"),
@@ -898,7 +898,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')")
           },
-          errorClass = "UNSUPPORTED_TABLE_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_VIEW_NOT_TABLE.USE_ALTER_TABLE",
           parameters = Map(
             "tableName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$tabName`",
             "operation" -> "ALTER VIEW ... UNSET TBLPROPERTIES"),
@@ -909,7 +909,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... UNSET TBLPROPERTIES"),
@@ -920,7 +920,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... SET LOCATION ..."),
@@ -931,7 +931,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET SERDE 'whatever'")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
@@ -942,7 +942,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
@@ -953,7 +953,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.USE_ALTER_VIEW",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... SET [SERDE|SERDEPROPERTIES]"),
@@ -964,7 +964,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName RECOVER PARTITIONS")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... RECOVER PARTITIONS"),
@@ -975,7 +975,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... RENAME TO PARTITION"),
@@ -986,7 +986,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... ADD PARTITION ..."),
@@ -997,7 +997,7 @@ class HiveDDLSuite
           exception = intercept[AnalysisException] {
             sql(s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')")
           },
-          errorClass = "UNSUPPORTED_VIEW_OPERATION.WITH_SUGGESTION",
+          errorClass = "EXPECT_TABLE_NOT_VIEW.NO_ALTERNATIVE",
           parameters = Map(
             "viewName" -> s"`$SESSION_CATALOG_NAME`.`default`.`$oldViewName`",
             "operation" -> "ALTER TABLE ... DROP PARTITION ..."),


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