diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 78269d29fa2cd..025a4ece8d1aa 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4252,12 +4252,6 @@ ], "sqlState" : "42883" }, - "UNSET_NONEXISTENT_PROPERTIES" : { - "message" : [ - "Attempted to unset non-existent properties [] in table ." - ], - "sqlState" : "42K0J" - }, "UNSUPPORTED_ADD_FILE" : { "message" : [ "Don't support add file." diff --git a/docs/sql-ref-syntax-ddl-alter-table.md b/docs/sql-ref-syntax-ddl-alter-table.md index 566e73da21513..31eaf659b5c7a 100644 --- a/docs/sql-ref-syntax-ddl-alter-table.md +++ b/docs/sql-ref-syntax-ddl-alter-table.md @@ -236,21 +236,29 @@ ALTER TABLE table_identifier DROP [ IF EXISTS ] partition_spec [PURGE] ### SET AND UNSET -#### SET TABLE PROPERTIES +#### SET PROPERTIES `ALTER TABLE SET` command is used for setting the table properties. If a particular property was already set, this overrides the old value with the new one. -`ALTER TABLE UNSET` is used to drop the table property. - ##### Syntax ```sql --- Set Table Properties +-- Set Properties ALTER TABLE table_identifier SET TBLPROPERTIES ( key1 = val1, key2 = val2, ... ) +``` + +#### UNSET PROPERTIES + +`ALTER TABLE UNSET` command is used to drop the table property. --- Unset Table Properties -ALTER TABLE table_identifier UNSET TBLPROPERTIES [ IF EXISTS ] ( key1, key2, ... ) +**Note:** If the specified property key does not exist, whether specify `IF EXISTS` or not, the command will ignore it and finally succeed. + +##### Syntax + +```sql +-- Unset Properties +ALTER TABLE table_identifier UNSET TBLPROPERTIES ( key1, key2, ... ) ``` #### SET SERDE diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index d3bd265d0459e..85556881184aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -2658,16 +2658,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "config" -> SQLConf.ALLOW_NON_EMPTY_LOCATION_IN_CTAS.key)) } - def unsetNonExistentPropertiesError( - properties: Seq[String], table: TableIdentifier): Throwable = { - new AnalysisException( - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - messageParameters = Map( - "properties" -> properties.map(toSQLId).mkString(", "), - "table" -> toSQLId(table.nameParts)) - ) - } - def alterTableChangeColumnNotSupportedForColumnTypeError( tableName: String, originColumn: StructField, 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 16bc751aab88a..70cfa93254190 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 @@ -1093,25 +1093,6 @@ class DDLParserSuite extends AnalysisTest { ifExists = true)) } - // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); - test("alter table: alter table properties") { - val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" - val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" - - comparePlans( - parsePlan(sql2_table), - UnsetTableProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", true), - Seq("comment", "test"), - ifExists = false)) - comparePlans( - parsePlan(sql3_table), - UnsetTableProperties( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", true), - Seq("comment", "test"), - ifExists = true)) - } - test("alter table: add column") { comparePlans( parsePlan("ALTER TABLE table_name ADD COLUMN x int"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index 6f402188910e0..3f221bfa53051 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -335,14 +335,6 @@ case class AlterTableUnsetPropertiesCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableRawMetadata(tableName) - if (!ifExists) { - val nonexistentKeys = propKeys.filter(key => !table.properties.contains(key) - && key != TableCatalog.PROP_COMMENT) - if (nonexistentKeys.nonEmpty) { - throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, table.identifier) - } - } // If comment is in the table property, we reset it to None val tableComment = if (propKeys.contains(TableCatalog.PROP_COMMENT)) None else table.comment val newProperties = table.properties.filter { case (k, _) => !propKeys.contains(k) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala index 64b70d709b93f..ac3c84dff718c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableSetTblPropertiesSuiteBase.scala @@ -19,6 +19,10 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.internal.SQLConf /** * This base suite contains unified tests for the `ALTER TABLE .. SET TBLPROPERTIES` @@ -39,7 +43,26 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU def checkTblProps(tableIdent: TableIdentifier, expectedTblProps: Map[String, String]): Unit - test("alter table set tblproperties") { + def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String + + test("table to alter does not exist") { + withNamespaceAndTable("ns", "does_not_exist") { t => + val sqlText = s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1')" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> toSQLId(t)), + context = ExpectedContext( + fragment = t, + start = 12, + stop = 11 + t.length) + ) + } + } + + test("alter table set properties") { withNamespaceAndTable("ns", "tbl") { t => sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog)) @@ -54,16 +77,53 @@ trait AlterTableSetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestU sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v8')") checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v8", "k3" -> "v3")) + } + } - // table to alter does not exist - checkError( - exception = intercept[AnalysisException] { - sql("ALTER TABLE does_not_exist SET TBLPROPERTIES ('winner' = 'loser')") - }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", - parameters = Map("relationName" -> "`does_not_exist`"), - context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25) - ) + test("alter table set reserved properties") { + import TableCatalog._ + val keyParameters = Map[String, String]( + PROP_PROVIDER -> "please use the USING clause to specify it", + PROP_LOCATION -> "please use the LOCATION clause to specify it", + PROP_OWNER -> "it will be set to the current user", + PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE" + ) + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespaceAndTable("ns", "tbl") { t => + val sqlText = s"ALTER TABLE $t SET TBLPROPERTIES ('$key'='bar')" + checkError( + exception = intercept[ParseException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + parameters = Map( + "property" -> key, + "msg" -> keyParameters.getOrElse( + key, "please remove it from the TBLPROPERTIES list.")), + context = ExpectedContext( + fragment = sqlText, + start = 0, + stop = 40 + t.length + key.length)) + } + } + } + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (key int) USING parquet $clause ('$key'='bar')") + val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog)) + + val originValue = getTblPropertyValue(tableIdent, key) + assert(originValue != "bar", "reserved properties should not have side effects") + + sql(s"ALTER TABLE $t SET TBLPROPERTIES ('$key'='newValue')") + assert(getTblPropertyValue(tableIdent, key) == originValue, + "reserved properties should not have side effects") + } + } + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala new file mode 100644 index 0000000000000..1e675a64f2235 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesParserSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.execution.command + +import org.apache.spark.SparkThrowable +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedTable} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.UnsetTableProperties +import org.apache.spark.sql.test.SharedSparkSession + +class AlterTableUnsetTblPropertiesParserSuite extends AnalysisTest with SharedSparkSession { + + private def parseException(sqlText: String): SparkThrowable = { + intercept[ParseException](sql(sqlText).collect()) + } + + // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter table: alter table properties") { + val sql1 = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql2 = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + comparePlans( + parsePlan(sql1), + UnsetTableProperties( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", + suggestAlternative = true), + Seq("comment", "test"), + ifExists = false)) + comparePlans( + parsePlan(sql2), + UnsetTableProperties( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... UNSET TBLPROPERTIES", + suggestAlternative = true), + Seq("comment", "test"), + ifExists = true)) + } + + test("alter table unset properties - property values must NOT be set") { + val sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" + checkError( + exception = parseException(sql), + errorClass = "_LEGACY_ERROR_TEMP_0035", + parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), + context = ExpectedContext( + fragment = sql, + start = 0, + stop = 80)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala new file mode 100644 index 0000000000000..be8d85d2ef670 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableUnsetTblPropertiesSuiteBase.scala @@ -0,0 +1,149 @@ +/* + * 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.execution.command + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, TableCatalog} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `ALTER TABLE .. UNSET TBLPROPERTIES` + * command that check V1 and V2 table catalogs. The tests that cannot run for all supported + * catalogs are located in more specific test suites: + * + * - V2 table catalog tests: + * `org.apache.spark.sql.execution.command.v2.AlterTableUnsetTblPropertiesSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterTableUnsetTblPropertiesSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterTableUnsetTblPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableUnsetTblPropertiesSuite` + */ +trait AlterTableUnsetTblPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER TABLE .. UNSET TBLPROPERTIES" + + def checkTblProps(tableIdent: TableIdentifier, expectedTblProps: Map[String, String]): Unit + + def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String + + test("table to alter does not exist") { + withNamespaceAndTable("ns", "does_not_exist") { t => + val sqlText = s"ALTER TABLE $t UNSET TBLPROPERTIES ('k1')" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> toSQLId(t)), + context = ExpectedContext( + fragment = t, + start = 12, + stop = 11 + t.length) + ) + } + } + + test("alter table unset properties") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") + val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog)) + checkTblProps(tableIdent, Map.empty[String, String]) + + sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v2', 'k3' = 'v3', 'k4' = 'v4')") + checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2", "k3" -> "v3", "k4" -> "v4")) + + // unset table properties + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('k1')") + checkTblProps(tableIdent, Map("k2" -> "v2", "k3" -> "v3", "k4" -> "v4")) + + // unset table properties without explicitly specifying database + sql(s"USE $catalog.ns") + sql(s"ALTER TABLE tbl UNSET TBLPROPERTIES ('k2')") + checkTblProps(tableIdent, Map("k3" -> "v3", "k4" -> "v4")) + } + } + + test("alter table unset non-existent properties") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") + val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog)) + + sql(s"ALTER TABLE $t SET TBLPROPERTIES ('k1' = 'v1', 'k2' = 'v2', 'k3' = 'v3')") + checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2", "k3" -> "v3")) + + // property to unset does not exist + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('k3', 'k4')") + checkTblProps(tableIdent, Map("k1" -> "v1", "k2" -> "v2")) + + // property to unset does not exist, but "IF EXISTS" is specified + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES IF EXISTS ('k2', 'k3')") + checkTblProps(tableIdent, Map("k1" -> "v1")) + } + } + + test("alter table unset reserved properties") { + import TableCatalog._ + val keyParameters = Map[String, String]( + PROP_PROVIDER -> "please use the USING clause to specify it", + PROP_LOCATION -> "please use the LOCATION clause to specify it", + PROP_OWNER -> "it will be set to the current user", + PROP_EXTERNAL -> "please use CREATE EXTERNAL TABLE" + ) + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespaceAndTable("ns", "tbl") { t => + val sqlText = s"ALTER TABLE $t UNSET TBLPROPERTIES ('$key')" + checkError( + exception = intercept[ParseException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_FEATURE.SET_TABLE_PROPERTY", + parameters = Map( + "property" -> key, + "msg" -> keyParameters.getOrElse( + key, "please remove it from the TBLPROPERTIES list.")), + context = ExpectedContext( + fragment = sqlText, + start = 0, + stop = 36 + t.length + key.length)) + } + } + } + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { + CatalogV2Util.TABLE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + Seq("OPTIONS", "TBLPROPERTIES").foreach { clause => + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (key int) USING parquet $clause ('$key'='bar')") + val tableIdent = TableIdentifier("tbl", Some("ns"), Some(catalog)) + + val originValue = getTblPropertyValue(tableIdent, key) + assert(originValue != "bar", "reserved properties should not have side effects") + + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('$key')") + assert(getTblPropertyValue(tableIdent, key) == originValue, + "reserved properties should not have side effects") + } + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index 505f0b4bdea62..70276051defa9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -107,18 +107,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { stop = 98)) } - test("alter table unset properties - property values must NOT be set") { - val sql = "ALTER TABLE my_tab UNSET TBLPROPERTIES('key_without_value', 'key_with_value'='x')" - checkError( - exception = parseException(sql), - errorClass = "_LEGACY_ERROR_TEMP_0035", - parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), - context = ExpectedContext( - fragment = sql, - start = 0, - stop = 80)) - } - test("alter table: exchange partition (not supported)") { val sql = """ALTER TABLE table_name_1 EXCHANGE PARTITION diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala index 553b68bec52fe..994c420feae1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala @@ -327,27 +327,6 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { protected val reversedProperties = Seq(PROP_OWNER) - test("alter table: unset properties (datasource table)") { - testUnsetProperties(isDatasourceTable = true) - } - - test("ALTER TABLE UNSET nonexistent property should throw an exception") { - val tableName = "test_table" - withTable(tableName) { - sql(s"CREATE TABLE $tableName (a STRING, b INT) USING parquet") - - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER TABLE $tableName UNSET TBLPROPERTIES ('test_prop1', 'test_prop2', 'comment')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map( - "properties" -> "`test_prop1`, `test_prop2`", - "table" -> "`spark_catalog`.`default`.`test_table`") - ) - } - } - test("alter table: change column (datasource table)") { testChangeColumn(isDatasourceTable = true) } @@ -1113,52 +1092,6 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { ) } - protected def testUnsetProperties(isDatasourceTable: Boolean): Unit = { - if (!isUsingHiveMetastore) { - assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") - } - val catalog = spark.sessionState.catalog - val tableIdent = TableIdentifier("tab1", Some("dbx")) - createDatabase(catalog, "dbx") - createTable(catalog, tableIdent, isDatasourceTable) - def getProps: Map[String, String] = { - if (isUsingHiveMetastore) { - normalizeCatalogTable(catalog.getTableMetadata(tableIdent)).properties - } else { - catalog.getTableMetadata(tableIdent).properties - } - } - // unset table properties - sql("ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('j' = 'am', 'p' = 'an', 'c' = 'lan', 'x' = 'y')") - sql("ALTER TABLE dbx.tab1 UNSET TBLPROPERTIES ('j')") - assert(getProps == Map("p" -> "an", "c" -> "lan", "x" -> "y")) - // unset table properties without explicitly specifying database - catalog.setCurrentDatabase("dbx") - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('p')") - assert(getProps == Map("c" -> "lan", "x" -> "y")) - // table to alter does not exist - val sql1 = "ALTER TABLE does_not_exist UNSET TBLPROPERTIES ('c' = 'lan')" - checkError( - exception = intercept[ParseException] { - sql(sql1) - }, - errorClass = "_LEGACY_ERROR_TEMP_0035", - parameters = Map("message" -> "Values should not be specified for key(s): [c]"), - context = ExpectedContext(fragment = sql1, start = 0, stop = 59) - ) - // property to unset does not exist - checkError( - exception = intercept[AnalysisException] { - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`xyz`", "table" -> "`spark_catalog`.`dbx`.`tab1`") - ) - // property to unset does not exist, but "IF EXISTS" is specified - sql("ALTER TABLE tab1 UNSET TBLPROPERTIES IF EXISTS ('c', 'xyz')") - assert(getProps == Map("x" -> "y")) - } - protected def testChangeColumn(isDatasourceTable: Boolean): Unit = { if (!isUsingHiveMetastore) { assert(isDatasourceTable, "InMemoryCatalog only supports data source tables") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetTblPropertiesSuite.scala index e74e5d4fc9ea5..cc59ee237daf3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetTblPropertiesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableSetTblPropertiesSuite.scala @@ -56,6 +56,10 @@ trait AlterTableSetTblPropertiesSuiteBase extends command.AlterTableSetTblProper assert(actualTblProps == expectedTblProps) } } + + override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = { + getTableProperties(tableIdent).getOrElse(key, null) + } } class AlterTableSetTblPropertiesSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableUnsetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableUnsetTblPropertiesSuite.scala new file mode 100644 index 0000000000000..0f7b4a0aad62e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableUnsetTblPropertiesSuite.scala @@ -0,0 +1,65 @@ +/* + * 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.execution.command.v1 + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION + +/** + * This base suite contains unified tests for the `ALTER TABLE .. UNSET TBLPROPERTIES` + * command that check V1 table catalogs. The tests that cannot run for all V1 catalogs + * are located in more specific test suites: + * + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterTableUnsetTblPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableUnsetTblPropertiesSuite` + */ +trait AlterTableUnsetTblPropertiesSuiteBase extends command.AlterTableUnsetTblPropertiesSuiteBase { + private[sql] lazy val sessionCatalog = spark.sessionState.catalog + + private def isUsingHiveMetastore: Boolean = { + spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive" + } + + private def normalizeTblProps(props: Map[String, String]): Map[String, String] = { + props.filterNot(p => Seq("transient_lastDdlTime").contains(p._1)) + } + + private def getTableProperties(tableIdent: TableIdentifier): Map[String, String] = { + sessionCatalog.getTableMetadata(tableIdent).properties + } + + override def checkTblProps(tableIdent: TableIdentifier, + expectedTblProps: Map[String, String]): Unit = { + val actualTblProps = getTableProperties(tableIdent) + if (isUsingHiveMetastore) { + assert(normalizeTblProps(actualTblProps) == expectedTblProps) + } else { + assert(actualTblProps == expectedTblProps) + } + } + + override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = { + getTableProperties(tableIdent).getOrElse(key, null) + } +} + +class AlterTableUnsetTblPropertiesSuite + extends AlterTableUnsetTblPropertiesSuiteBase with CommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetTblPropertiesSuite.scala index 7d7b2ad8686ee..22e19216d06f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetTblPropertiesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableSetTblPropertiesSuite.scala @@ -47,4 +47,8 @@ class AlterTableSetTblPropertiesSuite val actualTblProps = getTableMetadata(tableIdent).properties.asScala.toMap assert(normalizeTblProps(actualTblProps) === expectedTblProps) } + + override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = { + getTableMetadata(tableIdent).properties.asScala.toMap.getOrElse(key, null) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableUnsetTblPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableUnsetTblPropertiesSuite.scala new file mode 100644 index 0000000000000..699d8185ee6f0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableUnsetTblPropertiesSuite.scala @@ -0,0 +1,54 @@ +/* + * 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.execution.command.v2 + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.connector.catalog.{Identifier, Table} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +import org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `ALTER TABLE .. UNSET TBLPROPERTIES` command to + * check V2 table catalogs. + */ +class AlterTableUnsetTblPropertiesSuite + extends command.AlterTableUnsetTblPropertiesSuiteBase with CommandSuiteBase { + + private def normalizeTblProps(props: Map[String, String]): Map[String, String] = { + props.filterNot(p => Seq("provider", "owner").contains(p._1)) + } + + private def getTableMetadata(tableIndent: TableIdentifier): Table = { + val nameParts = tableIndent.nameParts + val v2Catalog = spark.sessionState.catalogManager.catalog(nameParts.head).asTableCatalog + val namespace = nameParts.drop(1).init.toArray + v2Catalog.loadTable(Identifier.of(namespace, nameParts.last)) + } + + override def checkTblProps(tableIdent: TableIdentifier, + expectedTblProps: Map[String, String]): Unit = { + val actualTblProps = getTableMetadata(tableIdent).properties.asScala.toMap + assert(normalizeTblProps(actualTblProps) === expectedTblProps) + } + + override def getTblPropertyValue(tableIdent: TableIdentifier, key: String): String = { + getTableMetadata(tableIdent).properties.asScala.toMap.getOrElse(key, null) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 65b70ad8bcaeb..fd437e7dc954f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -154,10 +154,6 @@ class HiveDDLSuite fs.exists(filesystemPath) } - test("alter table: unset properties") { - testUnsetProperties(isDatasourceTable = false) - } - test("alter table: change column") { testChangeColumn(isDatasourceTable = false) } @@ -767,15 +763,9 @@ class HiveDDLSuite sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") checkProperties(Map()) - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map( - "properties" -> "`p`", - "table" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") - ) + // unset non-existent properties + sql(s"ALTER VIEW $viewName UNSET TBLPROPERTIES ('p')") + checkProperties(Map()) } } } @@ -1884,16 +1874,6 @@ class HiveDDLSuite "tableName" -> "spark_catalog.default.tbl", "invalidKeys" -> s"[${forbiddenPrefix}foo]") ) - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER TABLE tbl UNSET TBLPROPERTIES ('${forbiddenPrefix}foo')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map( - "properties" -> (s"${(forbiddenPrefix.split("\\.") :+ "foo"). - map(part => s"`$part`").mkString(".")}"), - "table" -> "`spark_catalog`.`default`.`tbl`") - ) checkError( exception = intercept[AnalysisException] { sql(s"CREATE TABLE tbl2 (a INT) TBLPROPERTIES ('${forbiddenPrefix}foo'='anything')") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableUnsetTblPropertiesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableUnsetTblPropertiesSuite.scala new file mode 100644 index 0000000000000..d4e029eef2e02 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableUnsetTblPropertiesSuite.scala @@ -0,0 +1,27 @@ +/* + * 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.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for the `ALTER TABLE .. UNSET TBLPROPERTIES` command to check + * V1 Hive external table catalog. + */ +class AlterTableUnsetTblPropertiesSuite + extends v1.AlterTableUnsetTblPropertiesSuiteBase with CommandSuiteBase