Skip to content

Commit 2d81ba5

Browse files
gatorsmileyhuai
authored andcommitted
[SPARK-14362][SPARK-14406][SQL][FOLLOW-UP] DDL Native Support: Drop View and Drop Table
#### What changes were proposed in this pull request? In this PR, we are trying to address the comment in the original PR: dfce966#commitcomment-17057030 In this PR, we checks if table/view exists at the beginning and then does not need to capture the exceptions, including `NoSuchTableException` and `InvalidTableException`. We still capture the NonFatal exception when doing `sqlContext.cacheManager.tryUncacheQuery`. #### How was this patch tested? The existing test cases should cover the code changes of this PR. Author: gatorsmile <[email protected]> Closes #12321 from gatorsmile/dropViewFollowup.
1 parent 83fb964 commit 2d81ba5

File tree

1 file changed

+26
-24
lines changed
  • sql/core/src/main/scala/org/apache/spark/sql/execution/command

1 file changed

+26
-24
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala

Lines changed: 26 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql.execution.command
1919

20+
import scala.util.control.NonFatal
21+
2022
import org.apache.spark.internal.Logging
2123
import org.apache.spark.sql.{AnalysisException, Row, SQLContext}
2224
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -192,31 +194,31 @@ case class DropTable(
192194

193195
override def run(sqlContext: SQLContext): Seq[Row] = {
194196
val catalog = sqlContext.sessionState.catalog
195-
// If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view
196-
// issue an exception.
197-
catalog.getTableMetadataOption(tableName).map(_.tableType match {
198-
case CatalogTableType.VIRTUAL_VIEW if !isView =>
199-
throw new AnalysisException(
200-
"Cannot drop a view with DROP TABLE. Please use DROP VIEW instead")
201-
case o if o != CatalogTableType.VIRTUAL_VIEW && isView =>
202-
throw new AnalysisException(
203-
s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead")
204-
case _ =>
205-
})
206-
207-
try {
208-
sqlContext.cacheManager.tryUncacheQuery(sqlContext.table(tableName.quotedString))
209-
} catch {
210-
// This table's metadata is not in Hive metastore (e.g. the table does not exist).
211-
case e if e.getClass.getName == "org.apache.hadoop.hive.ql.metadata.InvalidTableException" =>
212-
case _: org.apache.spark.sql.catalyst.analysis.NoSuchTableException =>
213-
// Other Throwables can be caused by users providing wrong parameters in OPTIONS
214-
// (e.g. invalid paths). We catch it and log a warning message.
215-
// Users should be able to drop such kinds of tables regardless if there is an error.
216-
case e: Throwable => log.warn(s"${e.getMessage}", e)
197+
if (!catalog.tableExists(tableName)) {
198+
if (!ifExists) {
199+
val objectName = if (isView) "View" else "Table"
200+
logError(s"$objectName '${tableName.quotedString}' does not exist")
201+
}
202+
} else {
203+
// If the command DROP VIEW is to drop a table or DROP TABLE is to drop a view
204+
// issue an exception.
205+
catalog.getTableMetadataOption(tableName).map(_.tableType match {
206+
case CatalogTableType.VIRTUAL_VIEW if !isView =>
207+
throw new AnalysisException(
208+
"Cannot drop a view with DROP TABLE. Please use DROP VIEW instead")
209+
case o if o != CatalogTableType.VIRTUAL_VIEW && isView =>
210+
throw new AnalysisException(
211+
s"Cannot drop a table with DROP VIEW. Please use DROP TABLE instead")
212+
case _ =>
213+
})
214+
try {
215+
sqlContext.cacheManager.tryUncacheQuery(sqlContext.table(tableName.quotedString))
216+
} catch {
217+
case NonFatal(e) => log.warn(s"${e.getMessage}", e)
218+
}
219+
catalog.invalidateTable(tableName)
220+
catalog.dropTable(tableName, ifExists)
217221
}
218-
catalog.invalidateTable(tableName)
219-
catalog.dropTable(tableName, ifExists)
220222
Seq.empty[Row]
221223
}
222224
}

0 commit comments

Comments
 (0)