[SPARK-33174][SQL] Migrate DROP TABLE to use UnresolvedTableOrView to resolve the identifier

### What changes were proposed in this pull request?

This PR proposes to migrate `DROP TABLE` to use `UnresolvedTableOrView` to resolve the table/view identifier. This allows consistent resolution rules (temp view first, etc.) to be applied for both v1/v2 commands. More info about the consistent resolution rule proposal can be found in [JIRA](https://issues.apache.org/jira/browse/SPARK-29900) or [proposal doc](https://docs.google.com/document/d/1hvLjGA8y_W_hhilpngXVub1Ebv8RsMap986nENCFnrg/edit?usp=sharing).

### Why are the changes needed?

The current behavior is not consistent between v1 and v2 commands when resolving a temp view.
In v2, the `t` in the following example is resolved to a table:
```scala
sql("CREATE TABLE testcat.ns.t (id bigint) USING foo")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE testcat.ns")
sql("DROP TABLE t") // 't' is resolved to testcat.ns.t
```
whereas in v1, the `t` is resolved to a temp view:
```scala
sql("CREATE DATABASE test")
sql("CREATE TABLE spark_catalog.test.t (id bigint) USING csv")
sql("CREATE TEMPORARY VIEW t AS SELECT 2")
sql("USE spark_catalog.test")
sql("DROP TABLE t") // 't' is resolved to a temp view
```

### Does this PR introduce _any_ user-facing change?

After this PR, for v2, `DROP TABLE t` is resolved to a temp view `t` instead of `testcat.ns.t`, consistent with v1 behavior.

### How was this patch tested?

Added a new test

Closes #30079 from imback82/drop_table_consistent.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
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 52c96f4..61c077f 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
@@ -258,7 +258,9 @@
       ResolveUnion ::
       TypeCoercion.typeCoercionRules ++
       extendedResolutionRules : _*),
-    Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*),
+    Batch("Post-Hoc Resolution", Once,
+      Seq(ResolveNoopDropTable) ++
+      postHocResolutionRules: _*),
     Batch("Normalize Alter Table", Once, ResolveAlterTableChanges),
     Batch("Remove Unresolved Hints", Once,
       new ResolveHints.RemoveAllHints),
@@ -864,7 +866,9 @@
         }
         u
       case u @ UnresolvedTableOrView(ident) =>
-        lookupTempView(ident).map(_ => ResolvedView(ident.asIdentifier)).getOrElse(u)
+        lookupTempView(ident)
+          .map(_ => ResolvedView(ident.asIdentifier, isTemp = true))
+          .getOrElse(u)
     }
 
     def lookupTempView(
@@ -1017,7 +1021,8 @@
       case u @ UnresolvedTable(identifier) =>
         lookupTableOrView(identifier).map {
           case v: ResolvedView =>
-            u.failAnalysis(s"${v.identifier.quoted} is a view not table.")
+            val viewStr = if (v.isTemp) "temp view" else "view"
+            u.failAnalysis(s"${v.identifier.quoted} is a $viewStr not table.")
           case table => table
         }.getOrElse(u)
 
@@ -1030,7 +1035,7 @@
         case SessionCatalogAndIdentifier(catalog, ident) =>
           CatalogV2Util.loadTable(catalog, ident).map {
             case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW =>
-              ResolvedView(ident)
+              ResolvedView(ident, isTemp = false)
             case table =>
               ResolvedTable(catalog.asTableCatalog, ident, table)
           }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
index 65ddff8..d3bb72b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala
@@ -198,9 +198,6 @@
         writeOptions = c.writeOptions,
         orCreate = c.orCreate)
 
-    case DropTableStatement(NonSessionCatalogAndTable(catalog, tbl), ifExists, _) =>
-      DropTable(catalog.asTableCatalog, tbl.asIdentifier, ifExists)
-
     case DropViewStatement(NonSessionCatalogAndTable(catalog, viewName), _) =>
       throw new AnalysisException(
         s"Can not specify catalog `${catalog.name}` for view ${viewName.quoted} " +
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveNoopDropTable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveNoopDropTable.scala
new file mode 100644
index 0000000..f9da917
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveNoopDropTable.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.plans.logical.{DropTable, LogicalPlan, NoopDropTable}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+/**
+ * A rule for handling [[DropTable]] logical plan when the table or temp view is not resolved.
+ * If "ifExists" flag is set to true, the plan is resolved to [[NoopDropTable]],
+ * which is a no-op command.
+ */
+object ResolveNoopDropTable extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case DropTable(u: UnresolvedTableOrView, ifExists, _) if ifExists =>
+      NoopDropTable(u.multipartIdentifier)
+  }
+}
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 a16763f..1344d78 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
@@ -81,7 +81,7 @@
  */
 // TODO: create a generic representation for temp view, v1 view and v2 view, after we add view
 //       support to v2 catalog. For now we only need the identifier to fallback to v1 command.
-case class ResolvedView(identifier: Identifier) extends LeafNode {
+case class ResolvedView(identifier: Identifier, isTemp: Boolean) extends LeafNode {
   override def output: Seq[Attribute] = Nil
 }
 
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 f29e7b1..f28375c 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
@@ -2878,11 +2878,12 @@
   }
 
   /**
-   * Create a [[DropTableStatement]] command.
+   * Create a [[DropTable]] command.
    */
   override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) {
-    DropTableStatement(
-      visitMultipartIdentifier(ctx.multipartIdentifier()),
+    // DROP TABLE works with either a table or a temporary view.
+    DropTable(
+      UnresolvedTableOrView(visitMultipartIdentifier(ctx.multipartIdentifier())),
       ctx.EXISTS != null,
       ctx.PURGE != null)
   }
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
index d7c097a..3a534b2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala
@@ -284,14 +284,6 @@
     isView: Boolean) extends ParsedStatement
 
 /**
- * A DROP TABLE statement, as parsed from SQL.
- */
-case class DropTableStatement(
-    tableName: Seq[String],
-    ifExists: Boolean,
-    purge: Boolean) extends ParsedStatement
-
-/**
  * A DROP VIEW statement, as parsed from SQL.
  */
 case class DropViewStatement(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
index 272c19b..96cb096 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
@@ -388,9 +388,16 @@
  * The logical plan of the DROP TABLE command that works for v2 tables.
  */
 case class DropTable(
-    catalog: TableCatalog,
-    ident: Identifier,
-    ifExists: Boolean) extends Command
+    child: LogicalPlan,
+    ifExists: Boolean,
+    purge: Boolean) extends Command {
+  override def children: Seq[LogicalPlan] = child :: Nil
+}
+
+/**
+ * The logical plan for handling non-existing table for DROP TABLE command.
+ */
+case class NoopDropTable(multipartIdentifier: Seq[String]) extends Command
 
 /**
  * The logical plan of the ALTER TABLE command that works for v2 tables.
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 621d416..a81f9e1 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -381,19 +381,28 @@
 
   test("drop table") {
     parseCompare("DROP TABLE testcat.ns1.ns2.tbl",
-      DropTableStatement(Seq("testcat", "ns1", "ns2", "tbl"), ifExists = false, purge = false))
+      DropTable(
+        UnresolvedTableOrView(Seq("testcat", "ns1", "ns2", "tbl")),
+        ifExists = false,
+        purge = false))
     parseCompare(s"DROP TABLE db.tab",
-      DropTableStatement(Seq("db", "tab"), ifExists = false, purge = false))
+      DropTable(
+        UnresolvedTableOrView(Seq("db", "tab")), ifExists = false, purge = false))
     parseCompare(s"DROP TABLE IF EXISTS db.tab",
-      DropTableStatement(Seq("db", "tab"), ifExists = true, purge = false))
+      DropTable(
+        UnresolvedTableOrView(Seq("db", "tab")), ifExists = true, purge = false))
     parseCompare(s"DROP TABLE tab",
-      DropTableStatement(Seq("tab"), ifExists = false, purge = false))
+      DropTable(
+        UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = false))
     parseCompare(s"DROP TABLE IF EXISTS tab",
-      DropTableStatement(Seq("tab"), ifExists = true, purge = false))
+      DropTable(
+        UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = false))
     parseCompare(s"DROP TABLE tab PURGE",
-      DropTableStatement(Seq("tab"), ifExists = false, purge = true))
+      DropTable(
+        UnresolvedTableOrView(Seq("tab")), ifExists = false, purge = true))
     parseCompare(s"DROP TABLE IF EXISTS tab PURGE",
-      DropTableStatement(Seq("tab"), ifExists = true, purge = true))
+      DropTable(
+        UnresolvedTableOrView(Seq("tab")), ifExists = true, purge = true))
   }
 
   test("drop view") {
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 c4fd84c..f35eb41 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
@@ -260,14 +260,14 @@
       DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
     // Use v1 command to describe (temp) view, as v2 catalog doesn't support view yet.
-    case DescribeRelation(ResolvedView(ident), partitionSpec, isExtended) =>
+    case DescribeRelation(ResolvedView(ident, _), partitionSpec, isExtended) =>
       DescribeTableCommand(ident.asTableIdentifier, partitionSpec, isExtended)
 
     case DescribeColumn(r @ ResolvedTable(_, _, _: V1Table), colNameParts, isExtended)
         if isSessionCatalog(r.catalog) =>
       DescribeColumnCommand(r.identifier.asTableIdentifier, colNameParts, isExtended)
 
-    case DescribeColumn(ResolvedView(ident), colNameParts, isExtended) =>
+    case DescribeColumn(ResolvedView(ident, _), colNameParts, isExtended) =>
       DescribeColumnCommand(ident.asTableIdentifier, colNameParts, isExtended)
 
     // For CREATE TABLE [AS SELECT], we should use the v1 command if the catalog is resolved to the
@@ -367,9 +367,17 @@
           orCreate = c.orCreate)
       }
 
+    case DropTable(
+        r @ ResolvedTable(_, _, _: V1Table), ifExists, purge) if isSessionCatalog(r.catalog) =>
+      DropTableCommand(r.identifier.asTableIdentifier, ifExists, isView = false, purge = purge)
+
     // v1 DROP TABLE supports temp view.
-    case DropTableStatement(TempViewOrV1Table(name), ifExists, purge) =>
-      DropTableCommand(name.asTableIdentifier, ifExists, isView = false, purge = purge)
+    case DropTable(r: ResolvedView, ifExists, purge) =>
+      if (!r.isTemp) {
+        throw new AnalysisException(
+          "Cannot drop a view with DROP TABLE. Please use DROP VIEW instead")
+      }
+      DropTableCommand(r.identifier.asTableIdentifier, ifExists, isView = false, purge = purge)
 
     // v1 DROP TABLE supports temp view.
     case DropViewStatement(TempViewOrV1Table(name), ifExists) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
index 3841bd0..81a36de 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
@@ -26,7 +26,7 @@
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.connector.catalog.{CatalogV2Util, StagingTableCatalog, SupportsNamespaces, TableCapability, TableCatalog, TableChange}
 import org.apache.spark.sql.connector.read.streaming.{ContinuousStream, MicroBatchStream}
-import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, ProjectExec, RowDataSourceScanExec, SparkPlan}
+import org.apache.spark.sql.execution.{FilterExec, LeafExecNode, LocalTableScanExec, ProjectExec, RowDataSourceScanExec, SparkPlan}
 import org.apache.spark.sql.execution.datasources.DataSourceStrategy
 import org.apache.spark.sql.execution.streaming.continuous.{WriteToContinuousDataSource, WriteToContinuousDataSourceExec}
 import org.apache.spark.sql.sources.{BaseRelation, TableScan}
@@ -228,8 +228,11 @@
     case DescribeColumn(_: ResolvedTable, _, _) =>
       throw new AnalysisException("Describing columns is not supported for v2 tables.")
 
-    case DropTable(catalog, ident, ifExists) =>
-      DropTableExec(catalog, ident, ifExists) :: Nil
+    case DropTable(r: ResolvedTable, ifExists, _) =>
+      DropTableExec(r.catalog, r.identifier, ifExists) :: Nil
+
+    case NoopDropTable(multipartIdentifier) =>
+      LocalTableScanExec(Nil, Nil) :: Nil
 
     case AlterTable(catalog, ident, _, changes) =>
       AlterTableExec(catalog, ident, changes) :: Nil
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 e3618f1..298c070 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
@@ -744,10 +744,33 @@
   }
 
   test("DropTable: if exists") {
-    intercept[NoSuchTableException] {
-      sql(s"DROP TABLE testcat.db.notbl")
+    val ex = intercept[AnalysisException] {
+      sql("DROP TABLE testcat.db.notbl")
     }
-    sql(s"DROP TABLE IF EXISTS testcat.db.notbl")
+    assert(ex.getMessage.contains("Table or view not found: testcat.db.notbl"))
+    sql("DROP TABLE IF EXISTS testcat.db.notbl")
+  }
+
+  test("SPARK-33174: DROP TABLE should resolve to a temporary view first") {
+    withTable("testcat.ns.t") {
+      withTempView("t") {
+        sql("CREATE TABLE testcat.ns.t (id bigint) USING foo")
+        sql("CREATE TEMPORARY VIEW t AS SELECT 2")
+        sql("USE testcat.ns")
+
+        // Check the temporary view 't' exists.
+        runShowTablesSql(
+          "SHOW TABLES FROM spark_catalog.default LIKE 't'",
+          Seq(Row("", "t", true)),
+          expectV2Catalog = false)
+        sql("DROP TABLE t")
+        // Verify that the temporary view 't' is resolved first and dropped.
+        runShowTablesSql(
+          "SHOW TABLES FROM spark_catalog.default LIKE 't'",
+          Nil,
+          expectV2Catalog = false)
+      }
+    }
   }
 
   test("Relation: basic") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala
index 4e741ff..f57edb9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/TestV2SessionCatalogBase.scala
@@ -19,6 +19,7 @@
 
 import java.util
 import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.atomic.AtomicBoolean
 
 import scala.collection.JavaConverters._
 
@@ -36,6 +37,13 @@
 
   protected val tables: util.Map[Identifier, T] = new ConcurrentHashMap[Identifier, T]()
 
+  private val tableCreated: AtomicBoolean = new AtomicBoolean(false)
+
+  private def addTable(ident: Identifier, table: T): Unit = {
+    tableCreated.set(true)
+    tables.put(ident, table)
+  }
+
   protected def newTable(
       name: String,
       schema: StructType,
@@ -51,7 +59,7 @@
         case v1Table: V1Table if v1Table.v1Table.tableType == CatalogTableType.VIEW => v1Table
         case t =>
           val table = newTable(t.name(), t.schema(), t.partitioning(), t.properties())
-          tables.put(ident, table)
+          addTable(ident, table)
           table
       }
     }
@@ -64,7 +72,7 @@
       properties: util.Map[String, String]): Table = {
     val created = super.createTable(ident, schema, partitions, properties)
     val t = newTable(created.name(), schema, partitions, properties)
-    tables.put(ident, t)
+    addTable(ident, t)
     t
   }
 
@@ -74,8 +82,11 @@
   }
 
   def clearTables(): Unit = {
-    assert(!tables.isEmpty, "Tables were empty, maybe didn't use the session catalog code path?")
+    assert(
+      tableCreated.get,
+      "Tables are not created, maybe didn't use the session catalog code path?")
     tables.keySet().asScala.foreach(super.dropTable)
     tables.clear()
+    tableCreated.set(false)
   }
 }
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 8782295..d5820b0 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
@@ -630,10 +630,10 @@
   }
 
   test("drop table") {
-    val tableName1 = "db.tab"
-    val tableIdent1 = TableIdentifier("tab", Option("db"))
-    val tableName2 = "tab"
-    val tableIdent2 = TableIdentifier("tab", Some("default"))
+    val tableName1 = "db.v1Table"
+    val tableIdent1 = TableIdentifier("v1Table", Option("db"))
+    val tableName2 = "v1Table"
+    val tableIdent2 = TableIdentifier("v1Table", Some("default"))
 
     parseResolveCompare(s"DROP TABLE $tableName1",
       DropTableCommand(tableIdent1, ifExists = false, isView = false, purge = false))
@@ -656,13 +656,13 @@
     val tableIdent2 = Identifier.of(Array.empty, "tab")
 
     parseResolveCompare(s"DROP TABLE $tableName1",
-      DropTable(testCat, tableIdent1, ifExists = false))
+      DropTable(ResolvedTable(testCat, tableIdent1, table), ifExists = false, purge = false))
     parseResolveCompare(s"DROP TABLE IF EXISTS $tableName1",
-      DropTable(testCat, tableIdent1, ifExists = true))
+      DropTable(ResolvedTable(testCat, tableIdent1, table), ifExists = true, purge = false))
     parseResolveCompare(s"DROP TABLE $tableName2",
-      DropTable(testCat, tableIdent2, ifExists = false))
+      DropTable(ResolvedTable(testCat, tableIdent2, table), ifExists = false, purge = false))
     parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2",
-      DropTable(testCat, tableIdent2, ifExists = true))
+      DropTable(ResolvedTable(testCat, tableIdent2, table), ifExists = true, purge = false))
   }
 
   test("drop view") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala
index d99ccf8..51316b4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/jdbc/JDBCTableCatalogSuite.scala
@@ -78,10 +78,10 @@
     sql("DROP TABLE h2.test.to_drop")
     checkAnswer(sql("SHOW TABLES IN h2.test"), Seq(Row("test", "people")))
     Seq(
-      "h2.test.not_existing_table" -> "Table test.not_existing_table not found",
-      "h2.bad_test.not_existing_table" -> "Table bad_test.not_existing_table not found"
+      "h2.test.not_existing_table" -> "Table or view not found: h2.test.not_existing_table",
+      "h2.bad_test.not_existing_table" -> "Table or view not found: h2.bad_test.not_existing_table"
     ).foreach { case (table, expectedMsg) =>
-      val msg = intercept[NoSuchTableException] {
+      val msg = intercept[AnalysisException] {
         sql(s"DROP TABLE $table")
       }.getMessage
       assert(msg.contains(expectedMsg))