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/06/19 12:43:25 UTC

[spark] branch master updated: [SPARK-43203][SQL] Move all Drop Table case to DataSource V2

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 32a5db49e3e [SPARK-43203][SQL] Move all Drop Table case to DataSource V2
32a5db49e3e is described below

commit 32a5db49e3e62891736a8544dba440d5399a12c4
Author: Jia Fan <fa...@qq.com>
AuthorDate: Mon Jun 19 20:43:02 2023 +0800

    [SPARK-43203][SQL] Move all Drop Table case to DataSource V2
    
    <!--
    Thanks for sending a pull request!  Here are some tips for you:
      1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
      2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
      3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
      4. Be sure to keep the PR description updated to reflect all changes.
      5. Please write your PR title to summarize what this PR proposes.
      6. If possible, provide a concise example to reproduce the issue for a faster review.
      7. If you want to add a new configuration, please read the guideline first for naming configurations in
         'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
      8. If you want to add or modify an error type or message, please read the guideline first in
         'core/src/main/resources/error/README.md'.
    -->
    
    ### What changes were proposed in this pull request?
    In order to fix DROP table behavior in session catalog cause by #37879. Because we always invoke V1 drop logic if the identifier looks like a V1 identifier. This is a big blocker for external data sources that provide custom session catalogs.
    So this PR move all Drop Table case to DataSource V2 (use drop table to drop view not include). More information please check https://github.com/apache/spark/pull/37879/files#r1170501180
    
    <!--
    Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue.
    If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
      1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
      3. If you fix some SQL features, you can provide some references of other DBMSes.
      4. If there is design documentation, please add the link.
      5. If there is a discussion in the mailing list, please add the link.
    -->
    
    ### Why are the changes needed?
    Move Drop Table case to DataSource V2 to fix bug and prepare for remove drop table v1.
    <!--
    Please clarify why the changes are needed. For instance,
      1. If you propose a new API, clarify the use case for a new API.
      2. If you fix a bug, you can clarify why it is a bug.
    -->
    
    ### Does this PR introduce _any_ user-facing change?
    No
    <!--
    Note that it means *any* user-facing change including all aspects such as the documentation fix.
    If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
    If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
    If no, write 'No'.
    -->
    
    ### How was this patch tested?
    Tested by:
    - V2 table catalog tests: `org.apache.spark.sql.execution.command.v2.DropTableSuite`
    - V1 table catalog tests: `org.apache.spark.sql.execution.command.v1.DropTableSuiteBase`
    <!--
    If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
    If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
    If tests were not added, please describe why they were not added and/or why it was difficult to add.
    If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
    -->
    
    Closes #41348 from Hisoka-X/SPARK-43203_drop_table_to_v2.
    
    Authored-by: Jia Fan <fa...@qq.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../catalyst/analysis/ResolveSessionCatalog.scala  |  2 +-
 .../apache/spark/sql/execution/CacheManager.scala  |  6 ++++
 .../datasources/v2/V2SessionCatalog.scala          | 40 ++++++++++++++++-----
 .../ansi/conditional-functions.sql.out             |  3 +-
 .../ansi/decimalArithmeticOperations.sql.out       |  3 +-
 .../analyzer-results/change-column.sql.out         |  3 +-
 .../sql-tests/analyzer-results/charvarchar.sql.out | 12 ++++---
 .../columnresolution-negative.sql.out              |  6 ++--
 .../decimalArithmeticOperations.sql.out            |  3 +-
 .../describe-part-after-analyze.sql.out            |  3 +-
 .../analyzer-results/describe-query.sql.out        |  6 ++--
 .../describe-table-after-alter-table.sql.out       |  6 ++--
 .../sql-tests/analyzer-results/describe.sql.out    |  3 +-
 .../sql-tests/analyzer-results/explain-aqe.sql.out | 18 ++++++----
 .../sql-tests/analyzer-results/explain-cbo.sql.out |  6 ++--
 .../sql-tests/analyzer-results/explain.sql.out     | 18 ++++++----
 .../analyzer-results/identifier-clause.sql.out     | 15 +++++---
 .../analyzer-results/null-handling.sql.out         |  3 +-
 .../order-by-nulls-ordering.sql.out                |  6 ++--
 .../analyzer-results/postgreSQL/boolean.sql.out    | 12 ++++---
 .../analyzer-results/postgreSQL/case.sql.out       |  6 ++--
 .../postgreSQL/create_view.sql.out                 |  9 +++--
 .../analyzer-results/postgreSQL/date.sql.out       |  3 +-
 .../analyzer-results/postgreSQL/float4.sql.out     |  3 +-
 .../analyzer-results/postgreSQL/float8.sql.out     |  3 +-
 .../postgreSQL/groupingsets.sql.out                | 12 ++++---
 .../analyzer-results/postgreSQL/insert.sql.out     |  3 +-
 .../analyzer-results/postgreSQL/int2.sql.out       |  3 +-
 .../analyzer-results/postgreSQL/int4.sql.out       |  3 +-
 .../analyzer-results/postgreSQL/int8.sql.out       |  3 +-
 .../analyzer-results/postgreSQL/join.sql.out       | 21 +++++++----
 .../analyzer-results/postgreSQL/numeric.sql.out    | 42 ++++++++++++++--------
 .../analyzer-results/postgreSQL/select.sql.out     |  3 +-
 .../postgreSQL/select_having.sql.out               |  3 +-
 .../postgreSQL/select_implicit.sql.out             |  3 +-
 .../analyzer-results/postgreSQL/strings.sql.out    |  3 +-
 .../analyzer-results/postgreSQL/text.sql.out       |  3 +-
 .../analyzer-results/postgreSQL/timestamp.sql.out  |  3 +-
 .../postgreSQL/window_part2.sql.out                |  6 ++--
 .../postgreSQL/window_part3.sql.out                |  9 +++--
 .../analyzer-results/postgreSQL/with.sql.out       | 27 +++++++++-----
 .../analyzer-results/show-create-table.sql.out     | 33 +++++++++++------
 .../sql-tests/analyzer-results/show-tables.sql.out |  6 ++--
 .../analyzer-results/show-tblproperties.sql.out    |  3 +-
 .../analyzer-results/show_columns.sql.out          |  6 ++--
 .../udf/postgreSQL/udf-case.sql.out                |  6 ++--
 .../udf/postgreSQL/udf-join.sql.out                | 27 +++++++++-----
 .../udf/postgreSQL/udf-select_having.sql.out       |  3 +-
 .../udf/postgreSQL/udf-select_implicit.sql.out     |  3 +-
 .../execution/command/PlanResolutionSuite.scala    | 17 ++++-----
 .../hive/execution/command/DropTableSuite.scala    |  2 +-
 51 files changed, 304 insertions(+), 147 deletions(-)

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 fb1e9bcc591..d8e19c994c5 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
@@ -211,7 +211,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
         c
       }
 
-    case DropTable(ResolvedV1Identifier(ident), ifExists, purge) =>
+    case DropTable(ResolvedV1Identifier(ident), ifExists, purge) if conf.useV1Command =>
       DropTableCommand(ident, ifExists, isView = false, purge = purge)
 
     // v1 DROP TABLE supports temp view.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
index d41611439f0..b1153d7a1e8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config.ConfigEntry
 import org.apache.spark.sql.{Dataset, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
 import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression}
 import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint
 import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint, SubqueryAlias, View}
@@ -190,6 +191,11 @@ class CacheManager extends Logging with AdaptiveSparkPlanHelper {
         isSameName(ident.qualifier :+ ident.name) &&
           isSameName(v1Ident.catalog.toSeq ++ v1Ident.database :+ v1Ident.table)
 
+      case SubqueryAlias(ident, HiveTableRelation(catalogTable, _, _, _, _)) =>
+        val v1Ident = catalogTable.identifier
+        isSameName(ident.qualifier :+ ident.name) &&
+          isSameName(v1Ident.catalog.toSeq ++ v1Ident.database :+ v1Ident.table)
+
       case _ => false
     }
   }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
index 8234fb5a0b1..e5496a46760 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala
@@ -71,7 +71,12 @@ class V2SessionCatalog(catalog: SessionCatalog)
   }
 
   override def loadTable(ident: Identifier): Table = {
-    V1Table(catalog.getTableMetadata(ident.asTableIdentifier))
+    try {
+      V1Table(catalog.getTableMetadata(ident.asTableIdentifier))
+    } catch {
+      case _: NoSuchDatabaseException =>
+        throw QueryCompilationErrors.noSuchTableError(ident)
+    }
   }
 
   override def loadTable(ident: Identifier, timestamp: Long): Table = {
@@ -110,6 +115,10 @@ class V2SessionCatalog(catalog: SessionCatalog)
     createTable(ident, CatalogV2Util.v2ColumnsToStructType(columns), partitions, properties)
   }
 
+  override def purgeTable(ident: Identifier): Boolean = {
+    dropTableInternal(ident, purge = true)
+  }
+
   // TODO: remove it when no tests calling this deprecated method.
   override def createTable(
       ident: Identifier,
@@ -194,16 +203,29 @@ class V2SessionCatalog(catalog: SessionCatalog)
   }
 
   override def dropTable(ident: Identifier): Boolean = {
+    dropTableInternal(ident)
+  }
+
+  private def dropTableInternal(ident: Identifier, purge: Boolean = false): Boolean = {
     try {
-      if (loadTable(ident) != null) {
-        catalog.dropTable(
-          ident.asTableIdentifier,
-          ignoreIfNotExists = true,
-          purge = true /* skip HDFS trash */)
-        true
-      } else {
-        false
+      loadTable(ident) match {
+        case V1Table(v1Table) if v1Table.tableType == CatalogTableType.VIEW =>
+          throw QueryCompilationErrors.wrongCommandForObjectTypeError(
+            operation = "DROP TABLE",
+            requiredType = s"${CatalogTableType.EXTERNAL.name} or" +
+              s" ${CatalogTableType.MANAGED.name}",
+            objectName = v1Table.qualifiedName,
+            foundType = v1Table.tableType.name,
+            alternative = "DROP VIEW"
+          )
+        case _ =>
       }
+      catalog.invalidateCachedTable(ident.asTableIdentifier)
+      catalog.dropTable(
+        ident.asTableIdentifier,
+        ignoreIfNotExists = true,
+        purge = purge)
+      true
     } catch {
       case _: NoSuchTableException =>
         false
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out
index 0bd8648698b..38640878316 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/conditional-functions.sql.out
@@ -101,4 +101,5 @@ Project [CASE WHEN (1 > 2) THEN (cast(1 as double) / cast(0 as double)) ELSE cas
 -- !query
 DROP TABLE conditional_t
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`conditional_t`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.conditional_t
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out
index 8abc9148de0..bcaa991ddae 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/decimalArithmeticOperations.sql.out
@@ -166,4 +166,5 @@ Project [(10123456789012345678901234567890.123456 / 10.0) AS (101234567890123456
 -- !query
 drop table decimals_test
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`decimals_test`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.decimals_test
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 a996aaef4ca..0a1b745531c 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
@@ -255,7 +255,8 @@ org.apache.spark.sql.AnalysisException
 -- !query
 DROP TABLE test_change
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test_change`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_change
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
index 4e60588f9d4..61bac3e4d14 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/charvarchar.sql.out
@@ -659,25 +659,29 @@ Project [cast(c7#x as string) AS c7#x, cast(c8#x as string) AS c8#x, cast(v#x as
 -- !query
 drop table char_tbl1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`char_tbl1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.char_tbl1
 
 
 -- !query
 drop table char_tbl2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`char_tbl2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.char_tbl2
 
 
 -- !query
 drop table char_tbl3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`char_tbl3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.char_tbl3
 
 
 -- !query
 drop table char_tbl4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`char_tbl4`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.char_tbl4
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out
index c7ce2190199..61a366e862c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/columnresolution-negative.sql.out
@@ -395,7 +395,8 @@ CreateViewCommand `spark_catalog`.`mydb1`.`v1`, SELECT * FROM t1, false, false,
 -- !query
 DROP TABLE t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`mydb1`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), mydb1.t1
 
 
 -- !query
@@ -441,7 +442,8 @@ CreateViewCommand `v2`, SELECT * FROM t1, false, false, LocalTempView, true
 -- !query
 DROP TABLE t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`mydb2`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), mydb2.t1
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
index 31a46aa0399..2512a29b28a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/decimalArithmeticOperations.sql.out
@@ -442,4 +442,5 @@ Project [(10123456789012345678901234567890.123456 / 10.0) AS (101234567890123456
 -- !query
 drop table decimals_test
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`decimals_test`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.decimals_test
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
index 12e5edaddd2..8bb042e9979 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-part-after-analyze.sql.out
@@ -99,4 +99,5 @@ DescribeTableCommand `spark_catalog`.`default`.`t`, [ds=2017-09-01, hr=5], true,
 -- !query
 DROP TABLE t
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out
index c2a9f8e07bc..5119d6a08f2 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-query.sql.out
@@ -131,10 +131,12 @@ ExplainCommand DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT *
 -- !query
 DROP TABLE desc_temp1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`desc_temp1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.desc_temp1
 
 
 -- !query
 DROP TABLE desc_temp2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`desc_temp2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.desc_temp2
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out
index a23e3596922..af130710966 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe-table-after-alter-table.sql.out
@@ -26,7 +26,8 @@ DescribeTableCommand `spark_catalog`.`default`.`table_with_comment`, true, [col_
 -- !query
 DROP TABLE table_with_comment
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`table_with_comment`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_with_comment
 
 
 -- !query
@@ -68,4 +69,5 @@ DescribeTableCommand `spark_catalog`.`default`.`table_comment`, true, [col_name#
 -- !query
 DROP TABLE table_comment
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`table_comment`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.table_comment
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
index 9ea39f62b07..b9fe5c1d74c 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/describe.sql.out
@@ -293,7 +293,8 @@ ExplainCommand 'DescribeRelation [c=Us, d=2], false, [col_name#x, data_type#x, c
 -- !query
 DROP TABLE t
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out
index c53642b8ba2..f37e31bdb38 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-aqe.sql.out
@@ -202,31 +202,36 @@ ExplainCommand 'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], fal
 -- !query
 DROP TABLE explain_temp1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp1
 
 
 -- !query
 DROP TABLE explain_temp2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp2
 
 
 -- !query
 DROP TABLE explain_temp3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp3
 
 
 -- !query
 DROP TABLE explain_temp4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp4`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp4
 
 
 -- !query
 DROP TABLE explain_temp5
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp5`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp5
 
 
 -- !query
@@ -244,4 +249,5 @@ ExplainCommand 'Project [*], SimpleMode
 -- !query
 DROP TABLE t
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out
index 88b798b72e2..3375cdc10b9 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain-cbo.sql.out
@@ -46,10 +46,12 @@ ExplainCommand CTE [max_store_sales, best_ss_customer], CostMode
 -- !query
 DROP TABLE explain_temp1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp1
 
 
 -- !query
 DROP TABLE explain_temp2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp2
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out
index c53642b8ba2..f37e31bdb38 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/explain.sql.out
@@ -202,31 +202,36 @@ ExplainCommand 'InsertIntoStatement 'UnresolvedRelation [explain_temp5], [], fal
 -- !query
 DROP TABLE explain_temp1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp1
 
 
 -- !query
 DROP TABLE explain_temp2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp2
 
 
 -- !query
 DROP TABLE explain_temp3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp3
 
 
 -- !query
 DROP TABLE explain_temp4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp4`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp4
 
 
 -- !query
 DROP TABLE explain_temp5
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`explain_temp5`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.explain_temp5
 
 
 -- !query
@@ -244,4 +249,5 @@ ExplainCommand 'Project [*], SimpleMode
 -- !query
 DROP TABLE t
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
index 7c98d4d1670..7fc18dfe04e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out
@@ -168,7 +168,8 @@ SetNamespaceCommand [default]
 -- !query
 DROP TABLE s.tab
 -- !query analysis
-DropTableCommand `spark_catalog`.`s`.`tab`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), s.tab
 
 
 -- !query
@@ -208,7 +209,8 @@ CreateDataSourceTableCommand `spark_catalog`.`default`.`tab`, false
 -- !query
 DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b')
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tab`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab
 
 
 -- !query
@@ -234,7 +236,8 @@ CreateDataSourceTableCommand `spark_catalog`.`identifier_clauses`.`tab`, false
 -- !query
 DROP TABLE IF EXISTS IDENTIFIER('identifier_clauses.' || 'tab')
 -- !query analysis
-DropTableCommand `spark_catalog`.`identifier_clauses`.`tab`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab
 
 
 -- !query
@@ -276,7 +279,8 @@ UncacheTable false, true
 -- !query
 DROP TABLE IF EXISTS IDENTIFIER('ta' || 'b')
 -- !query analysis
-DropTableCommand `spark_catalog`.`identifier_clauses`.`tab`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), identifier_clauses.tab
 
 
 -- !query
@@ -381,7 +385,8 @@ TruncateTableCommand `spark_catalog`.`default`.`tab`
 -- !query
 DROP TABLE IF EXISTS tab
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tab`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tab
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
index 02807140ed3..26e9394932a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
@@ -236,4 +236,5 @@ Project [b#x, c#x, equal_null(b#x, c#x) AS equal_null(b, c)#x, equal_null(c#x, b
 -- !query
 drop table t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out
index 61c9c9578dd..1eb699deac7 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/order-by-nulls-ordering.sql.out
@@ -120,7 +120,8 @@ Sort [COL3#x DESC NULLS LAST, COL2#x ASC NULLS FIRST], true
 -- !query
 drop table spark_10747
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`spark_10747`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.spark_10747
 
 
 -- !query
@@ -181,4 +182,5 @@ Sort [col5#x DESC NULLS FIRST, col3#x DESC NULLS LAST], true
 -- !query
 drop table spark_10747_mix
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`spark_10747_mix`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.spark_10747_mix
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
index 0ec0b55bb27..355ccef60ac 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/boolean.sql.out
@@ -744,22 +744,26 @@ Project [((isnul#x OR istrue#x) OR isfalse#x) AS ((isnul OR istrue) OR isfalse)#
 -- !query
 DROP TABLE  BOOLTBL1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`BOOLTBL1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.BOOLTBL1
 
 
 -- !query
 DROP TABLE  BOOLTBL2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`BOOLTBL2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.BOOLTBL2
 
 
 -- !query
 DROP TABLE  BOOLTBL3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`BOOLTBL3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.BOOLTBL3
 
 
 -- !query
 DROP TABLE  BOOLTBL4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`BOOLTBL4`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.BOOLTBL4
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
index 80b658f1d1e..31d199e4754 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/case.sql.out
@@ -339,10 +339,12 @@ Project [CASE WHEN (CASE WHEN (vol(bar) = foo) THEN it was foo! WHEN (vol(bar) =
 -- !query
 DROP TABLE CASE_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`CASE_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.CASE_TBL
 
 
 -- !query
 DROP TABLE CASE2_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.CASE2_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
index 6c4fbfe6f79..ac6ee2af69d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/create_view.sql.out
@@ -31,7 +31,8 @@ DropTableCommand `spark_catalog`.`default`.`toyemp`, false, true, false
 -- !query
 DROP TABLE emp
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`emp`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.emp
 
 
 -- !query
@@ -207,7 +208,8 @@ DropTableCommand `spark_catalog`.`default`.`viewtest`, false, true, false
 -- !query
 DROP TABLE viewtest_tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`viewtest_tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.viewtest_tbl
 
 
 -- !query
@@ -1356,7 +1358,8 @@ DescribeTableCommand `spark_catalog`.`testviewschm2`.`v3`, false, [col_name#x, d
 -- !query
 drop table tt3
 -- !query analysis
-DropTableCommand `spark_catalog`.`testviewschm2`.`tt3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), testviewschm2.tt3
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
index 3f451a4d365..00813e42d7a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/date.sql.out
@@ -580,4 +580,5 @@ Project [make_date(2013, 11, -1, true) AS make_date(2013, 11, -1)#x]
 -- !query
 DROP TABLE DATE_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`DATE_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.DATE_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
index 22dd493eb91..0618a1d8432 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float4.sql.out
@@ -336,4 +336,5 @@ Project [cast(cast(-9223380000000000000 as float) as bigint) AS -922338000000000
 -- !query
 DROP TABLE FLOAT4_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`FLOAT4_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.FLOAT4_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
index b8861f07201..d30f971628e 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/float8.sql.out
@@ -743,4 +743,5 @@ Project [cast(cast(-9223372036854780000 as double) as bigint) AS -92233720368547
 -- !query
 DROP TABLE FLOAT8_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`FLOAT8_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.FLOAT8_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
index f7fd784d9f9..d8b7be296ff 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/groupingsets.sql.out
@@ -663,22 +663,26 @@ DropTempViewCommand gstest1
 -- !query
 DROP TABLE gstest2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`gstest2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.gstest2
 
 
 -- !query
 DROP TABLE gstest3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`gstest3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.gstest3
 
 
 -- !query
 DROP TABLE gstest4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`gstest4`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.gstest4
 
 
 -- !query
 DROP TABLE gstest_empty
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`gstest_empty`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.gstest_empty
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out
index 3d96f8aa641..9f232433e18 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/insert.sql.out
@@ -63,4 +63,5 @@ Project [col1#x, col2#x, char_length(col3#x) AS char_length(col3)#x]
 -- !query
 drop table inserttest
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`inserttest`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.inserttest
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
index 58c8904e91d..9dda3c0dc42 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int2.sql.out
@@ -327,4 +327,5 @@ Project [x#x, cast(x#x as smallint) AS int2_value#x]
 -- !query
 DROP TABLE INT2_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`INT2_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.INT2_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
index 9885ef0ef91..d261b59a4c5 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int4.sql.out
@@ -471,4 +471,5 @@ Project [x#x, cast(x#x as int) AS int4_value#x]
 -- !query
 DROP TABLE INT4_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`INT4_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.INT4_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
index b1faa6ee953..c7f3f7bdbbb 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/int8.sql.out
@@ -768,4 +768,5 @@ Project [x#x, cast(x#x as bigint) AS int8_value#xL]
 -- !query
 DROP TABLE INT8_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`INT8_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.INT8_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
index 2e747f52a7d..511c0b397be 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/join.sql.out
@@ -1586,31 +1586,36 @@ Aggregate [count(1) AS count(1)#xL]
 -- !query
 DROP TABLE t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
 
 
 -- !query
 DROP TABLE t2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
 
 
 -- !query
 DROP TABLE t3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t3
 
 
 -- !query
 DROP TABLE J1_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`J1_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.J1_TBL
 
 
 -- !query
 DROP TABLE J2_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`J2_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.J2_TBL
 
 
 -- !query
@@ -3558,10 +3563,12 @@ Project [id1#x, id2#x, id1#x, id2#x]
 -- !query
 drop table j1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`j1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.j1
 
 
 -- !query
 drop table j2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`j2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.j2
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
index a6408f94579..95c81ff75ed 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/numeric.sql.out
@@ -4205,7 +4205,8 @@ Project [id#x, val#x]
 -- !query
 DROP TABLE fract_only
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`fract_only`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.fract_only
 
 
 -- !query
@@ -4323,7 +4324,8 @@ Project [a#x, CEIL(a#x) AS CEIL(a)#x, ceiling(a#x) AS ceiling(a)#x, FLOOR(a#x) A
 -- !query
 DROP TABLE ceil_floor_round
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`ceil_floor_round`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.ceil_floor_round
 
 
 -- !query
@@ -4462,7 +4464,8 @@ Project [width_bucket(cast(Infinity as double), cast(1 as double), cast(10 as do
 -- !query
 DROP TABLE width_bucket_test
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`width_bucket_test`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.width_bucket_test
 
 
 -- !query
@@ -4905,64 +4908,75 @@ Aggregate [sum(cast(-9999 as decimal(10,0))) AS sum(-9999)#x]
 -- !query
 DROP TABLE num_data
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_data`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_data
 
 
 -- !query
 DROP TABLE num_exp_add
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_add`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_add
 
 
 -- !query
 DROP TABLE num_exp_sub
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_sub`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_sub
 
 
 -- !query
 DROP TABLE num_exp_div
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_div`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_div
 
 
 -- !query
 DROP TABLE num_exp_mul
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_mul`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_mul
 
 
 -- !query
 DROP TABLE num_exp_sqrt
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_sqrt`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_sqrt
 
 
 -- !query
 DROP TABLE num_exp_ln
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_ln`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_ln
 
 
 -- !query
 DROP TABLE num_exp_log10
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_log10`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_log10
 
 
 -- !query
 DROP TABLE num_exp_power_10_ln
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_exp_power_10_ln`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_exp_power_10_ln
 
 
 -- !query
 DROP TABLE num_result
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_result`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_result
 
 
 -- !query
 DROP TABLE num_input_test
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`num_input_test`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.num_input_test
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out
index 3a0b4c03666..9703c5dbc2d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select.sql.out
@@ -455,4 +455,5 @@ Project [k#x]
 -- !query
 drop table tmp
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tmp`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tmp
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
index a841d56b346..15df3507060 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_having.sql.out
@@ -208,4 +208,5 @@ Filter (one#x < 2)
 -- !query
 DROP TABLE test_having
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test_having`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_having
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
index 417b433b5d9..687eeee8f6a 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/select_implicit.sql.out
@@ -444,4 +444,5 @@ org.apache.spark.sql.AnalysisException
 -- !query
 DROP TABLE test_missing_target
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test_missing_target`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_missing_target
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
index ab078642e96..2a4d79360b5 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/strings.sql.out
@@ -874,4 +874,5 @@ Project [btrim(cast(\000trim\000 as binary), cast( as binary)) AS btrim(\000trim
 -- !query
 DROP TABLE toasttest
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`toasttest`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.toasttest
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
index 6e0345c2955..7fb0649f5e7 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/text.sql.out
@@ -314,4 +314,5 @@ Project [format_string(>>%1$10s<<, Hello) AS format_string(>>%1$10s<<, Hello)#x]
 -- !query
 DROP TABLE TEXT_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`TEXT_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.TEXT_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
index 2feefcf9745..cc30ebddcbb 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/timestamp.sql.out
@@ -220,4 +220,5 @@ Project [make_timestamp(2014, 12, 28, 6, 30, cast(45.887 as decimal(16,6)), None
 -- !query
 DROP TABLE TIMESTAMP_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`TIMESTAMP_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.TIMESTAMP_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
index e8d8c069276..dfc4fdde717 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part2.sql.out
@@ -467,10 +467,12 @@ org.apache.spark.SparkNumberFormatException
 -- !query
 drop table empsalary
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`empsalary`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.empsalary
 
 
 -- !query
 drop table numerics
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`numerics`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.numerics
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
index 52b58a4a296..53927b549bb 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out
@@ -473,16 +473,19 @@ org.apache.spark.sql.AnalysisException
 -- !query
 DROP TABLE empsalary
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`empsalary`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.empsalary
 
 
 -- !query
 DROP TABLE datetimes
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`datetimes`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.datetimes
 
 
 -- !query
 DROP TABLE t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
index bf37c45b0be..e53480e96be 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out
@@ -161,7 +161,8 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
 -- !query
 DROP TABLE y
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`y`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.y
 
 
 -- !query
@@ -279,7 +280,8 @@ Sort [k#x ASC NULLS FIRST], true
 -- !query
 DROP TABLE withz
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`withz`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.withz
 
 
 -- !query
@@ -467,40 +469,47 @@ Project [i#x]
 -- !query
 drop table test
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test
 
 
 -- !query
 DROP TABLE department
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`department`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.department
 
 
 -- !query
 DROP TABLE tree
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tree`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tree
 
 
 -- !query
 DROP TABLE graph
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`graph`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.graph
 
 
 -- !query
 DROP TABLE y
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`y`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.y
 
 
 -- !query
 DROP TABLE yy
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`yy`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.yy
 
 
 -- !query
 DROP TABLE parent
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`parent`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.parent
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out
index bfc49ef55cc..b44edb8dc4d 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-create-table.sql.out
@@ -15,7 +15,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -35,7 +36,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -55,7 +57,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -75,7 +78,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -95,7 +99,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -115,7 +120,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -135,7 +141,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -155,7 +162,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -175,7 +183,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -194,7 +203,8 @@ ShowCreateTable false, [createtab_stmt#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
@@ -292,4 +302,5 @@ DropTableCommand `spark_catalog`.`default`.`view_SPARK_30302`, false, true, fals
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
index e59436be9ce..167e2f8622b 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tables.sql.out
@@ -184,13 +184,15 @@ org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
 -- !query
 DROP TABLE show_t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`showdb`.`show_t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), showdb.show_t1
 
 
 -- !query
 DROP TABLE show_t2
 -- !query analysis
-DropTableCommand `spark_catalog`.`showdb`.`show_t2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), showdb.show_t2
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out
index a9ecb72ba6f..0ea52d7d1e8 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-tblproperties.sql.out
@@ -30,7 +30,8 @@ ShowTableProperties p3, [key#x, value#x]
 -- !query
 DROP TABLE tbl
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tbl`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tbl
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out
index b174a5870f6..3c790c07891 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show_columns.sql.out
@@ -183,13 +183,15 @@ ShowColumnsCommand global_temp, `global_temp`.`showcolumn4`, [col_name#x]
 -- !query
 DROP TABLE showcolumn1
 -- !query analysis
-DropTableCommand `spark_catalog`.`showdb`.`showcolumn1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), showdb.showcolumn1
 
 
 -- !query
 DROP TABLE showColumn2
 -- !query analysis
-DropTableCommand `spark_catalog`.`showdb`.`showColumn2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), showdb.showColumn2
 
 
 -- !query
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out
index 5502e898ae6..c0f00168908 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-case.sql.out
@@ -331,10 +331,12 @@ Project [CASE WHEN (CASE WHEN (vol(bar) = cast(udf(cast(foo as string)) as strin
 -- !query
 DROP TABLE CASE_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`CASE_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.CASE_TBL
 
 
 -- !query
 DROP TABLE CASE2_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`CASE2_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.CASE2_TBL
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out
index 239216f760b..83395e9e9b1 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-join.sql.out
@@ -1590,31 +1590,36 @@ Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL]
 -- !query
 DROP TABLE t1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t1
 
 
 -- !query
 DROP TABLE t2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2
 
 
 -- !query
 DROP TABLE t3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`t3`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t3
 
 
 -- !query
 DROP TABLE J1_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`J1_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.J1_TBL
 
 
 -- !query
 DROP TABLE J2_TBL
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`J2_TBL`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.J2_TBL
 
 
 -- !query
@@ -1705,7 +1710,8 @@ Aggregate [cast(udf(cast(count(1) as string)) as bigint) AS udf(count(1))#xL]
 -- !query
 DROP TABLE IF EXISTS tt3
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tt3`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tt3
 
 
 -- !query
@@ -1727,7 +1733,8 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d
 -- !query
 DROP TABLE IF EXISTS tt4
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`tt4`, true, false, false
+DropTable true, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.tt4
 
 
 -- !query
@@ -3556,10 +3563,12 @@ Project [id1#x, id2#x, id1#x, id2#x]
 -- !query
 drop table j1
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`j1`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.j1
 
 
 -- !query
 drop table j2
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`j2`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.j2
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out
index b864765b0e8..065a0c65a40 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_having.sql.out
@@ -212,4 +212,5 @@ Filter (one#x < 2)
 -- !query
 DROP TABLE test_having
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test_having`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_having
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out
index 5ce91e5690a..2377140ad92 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/udf/postgreSQL/udf-select_implicit.sql.out
@@ -449,4 +449,5 @@ org.apache.spark.sql.AnalysisException
 -- !query
 DROP TABLE test_missing_target
 -- !query analysis
-DropTableCommand `spark_catalog`.`default`.`test_missing_target`, false, false, false
+DropTable false, false
++- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.test_missing_target
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
index 17a0f308a1a..ee1c70a05d6 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
@@ -691,22 +691,23 @@ class PlanResolutionSuite extends AnalysisTest {
 
   test("drop table") {
     val tableName1 = "db.v1Table"
-    val tableIdent1 = TableIdentifier("v1Table", Option("db"), Some(SESSION_CATALOG_NAME))
+    val tableIdent1 = ResolvedIdentifier(v2SessionCatalog, Identifier.of(Array("db"), "v1Table"))
     val tableName2 = "v1Table"
-    val tableIdent2 = TableIdentifier("v1Table", Some("default"), Some(SESSION_CATALOG_NAME))
+    val tableIdent2 = ResolvedIdentifier(v2SessionCatalog, Identifier.of(Array("default"),
+      "v1Table"))
 
     parseResolveCompare(s"DROP TABLE $tableName1",
-      DropTableCommand(tableIdent1, ifExists = false, isView = false, purge = false))
+      DropTable(tableIdent1, ifExists = false, purge = false))
     parseResolveCompare(s"DROP TABLE IF EXISTS $tableName1",
-      DropTableCommand(tableIdent1, ifExists = true, isView = false, purge = false))
+      DropTable(tableIdent1, ifExists = true, purge = false))
     parseResolveCompare(s"DROP TABLE $tableName2",
-      DropTableCommand(tableIdent2, ifExists = false, isView = false, purge = false))
+      DropTable(tableIdent2, ifExists = false, purge = false))
     parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2",
-      DropTableCommand(tableIdent2, ifExists = true, isView = false, purge = false))
+      DropTable(tableIdent2, ifExists = true, purge = false))
     parseResolveCompare(s"DROP TABLE $tableName2 PURGE",
-      DropTableCommand(tableIdent2, ifExists = false, isView = false, purge = true))
+      DropTable(tableIdent2, ifExists = false, purge = true))
     parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2 PURGE",
-      DropTableCommand(tableIdent2, ifExists = true, isView = false, purge = true))
+      DropTable(tableIdent2, ifExists = true, purge = true))
   }
 
   test("drop table in v2 catalog") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala
index 8c6d718f18a..e847e92c4ce 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/DropTableSuite.scala
@@ -26,7 +26,7 @@ class DropTableSuite extends v1.DropTableSuiteBase with CommandSuiteBase {
   test("hive client calls") {
     withNamespaceAndTable("ns", "tbl") { t =>
       sql(s"CREATE TABLE $t (id int) $defaultUsing")
-      checkHiveClientCalls(expected = 11) {
+      checkHiveClientCalls(expected = 10) {
         sql(s"DROP TABLE $t")
       }
     }


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