diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 01cd181010f97..606395fd6fec2 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -207,7 +207,7 @@ statement | REFRESH (STRING | .*?) #refreshResource | CACHE LAZY? TABLE multipartIdentifier (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable - | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable + | UNCACHE TABLE (IF EXISTS)? multipartIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE tableIdentifier partitionSpec? #loadData 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 99e5c9feb8fa3..d2c23271a6277 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 @@ -2793,6 +2793,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CacheTableStatement(tableName, query, ctx.LAZY != null, options) } + /** + * Create an [[UncacheTableStatement]] logical plan. + */ + override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { + UncacheTableStatement(visitMultipartIdentifier(ctx.multipartIdentifier), ctx.EXISTS != null) + } + /** * Create a [[TruncateTableStatement]] command. * 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 4a91ee6d52d9e..ef8c922694347 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 @@ -339,6 +339,13 @@ case class CacheTableStatement( isLazy: Boolean, options: Map[String, String]) extends ParsedStatement +/** + * An UNCACHE TABLE statement, as parsed from SQL + */ +case class UncacheTableStatement( + tableName: Seq[String], + ifExists: Boolean) extends ParsedStatement + /** * A TRUNCATE TABLE statement, as parsed from SQL */ 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 37349f7a33420..f4375956f0af6 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 @@ -1039,13 +1039,13 @@ class DDLParserSuite extends AnalysisTest { "missing 'COLUMNS' at ''") } - test("MSCK REPAIR table") { + test("MSCK REPAIR TABLE") { comparePlans( parsePlan("MSCK REPAIR TABLE a.b.c"), RepairTableStatement(Seq("a", "b", "c"))) } - test("CACHE table") { + test("CACHE TABLE") { comparePlans( parsePlan("CACHE TABLE a.b.c"), CacheTableStatement(Seq("a", "b", "c"), None, false, Map.empty)) @@ -1062,6 +1062,16 @@ class DDLParserSuite extends AnalysisTest { "It is not allowed to add catalog/namespace prefix a.b") } + test("UNCACHE TABLE") { + comparePlans( + parsePlan("UNCACHE TABLE a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = false)) + + comparePlans( + parsePlan("UNCACHE TABLE IF EXISTS a.b.c"), + UncacheTableStatement(Seq("a", "b", "c"), ifExists = true)) + } + test("TRUNCATE table") { comparePlans( parsePlan("TRUNCATE TABLE a.b.c"), @@ -1098,7 +1108,7 @@ class DDLParserSuite extends AnalysisTest { comparePlans(parsed5, expected5) } - test("REFRESH TABLE table") { + test("REFRESH TABLE") { comparePlans( parsePlan("REFRESH TABLE a.b.c"), RefreshTableStatement(Seq("a", "b", "c"))) 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 65d95b600eaa2..f91686cb544c9 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog, TableChange, V1Table} import org.apache.spark.sql.connector.expressions.Transform -import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand} +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableRecoverPartitionsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, AnalyzeColumnCommand, AnalyzePartitionCommand, AnalyzeTableCommand, CacheTableCommand, CreateDatabaseCommand, DescribeColumnCommand, DescribeTableCommand, DropTableCommand, ShowPartitionsCommand, ShowTablesCommand, TruncateTableCommand, UncacheTableCommand} import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, RefreshTable} import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf @@ -303,6 +303,10 @@ class ResolveSessionCatalog( val v1TableName = parseV1Table(tableName, "CACHE TABLE") CacheTableCommand(v1TableName.asTableIdentifier, plan, isLazy, options) + case UncacheTableStatement(tableName, ifExists) => + val v1TableName = parseV1Table(tableName, "UNCACHE TABLE") + UncacheTableCommand(v1TableName.asTableIdentifier, ifExists) + case TruncateTableStatement(tableName, partitionSpec) => val v1TableName = parseV1Table(tableName, "TRUNCATE TABLE") TruncateTableCommand( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index fb13d01bd91dc..aef0a2d2e5951 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -165,13 +165,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { unquotedPath } - /** - * Create an [[UncacheTableCommand]] logical plan. - */ - override def visitUncacheTable(ctx: UncacheTableContext): LogicalPlan = withOrigin(ctx) { - UncacheTableCommand(visitTableIdentifier(ctx.tableIdentifier), ctx.EXISTS != null) - } - /** * Create a [[ClearCacheCommand]] logical plan. */ 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 4d1e70f68ba06..4f2c1af8f7b56 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 @@ -1282,6 +1282,16 @@ class DataSourceV2SQLSuite } } + test("UNCACHE TABLE") { + val t = "testcat.ns1.ns2.tbl" + withTable(t) { + sql(s"CREATE TABLE $t (id bigint, data string) USING foo") + + testV1Command("UNCACHE TABLE", t) + testV1Command("UNCACHE TABLE", s"IF EXISTS $t") + } + } + private def testV1Command(sqlCommand: String, sqlParams: String): Unit = { val e = intercept[AnalysisException] { sql(s"$sqlCommand $sqlParams")