From 12f8f71ee23796a587d5d1bfb575e006ee9c2bd0 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 20 Jun 2024 16:13:48 +0800 Subject: [PATCH 1/9] [SPARK-48668][SQL] Support ALTER NAMESPACE ... UNSET PROPERTIES in v2 --- .../resources/error/error-conditions.json | 2 +- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 + .../sql/catalyst/parser/AstBuilder.scala | 20 +++ .../catalyst/plans/logical/v2Commands.scala | 17 +++ .../sql/errors/QueryCompilationErrors.scala | 4 +- .../spark/sql/execution/command/ddl.scala | 2 +- .../AlterNamespaceUnsetPropertiesExec.scala | 50 +++++++ .../datasources/v2/DataSourceV2Strategy.scala | 3 + ...rNamespaceUnsetPropertiesParserSuite.scala | 57 ++++++++ ...terNamespaceUnsetPropertiesSuiteBase.scala | 127 ++++++++++++++++++ .../sql/execution/command/DDLSuite.scala | 4 +- ...terNamespaceUnsetPropertiesSuiteBase.scala | 45 +++++++ .../AlterNamespaceUnsetPropertiesSuite.scala | 30 +++++ .../sql/hive/execution/HiveDDLSuite.scala | 4 +- .../AlterNamespaceUnsetPropertiesSuite.scala | 29 ++++ 15 files changed, 388 insertions(+), 8 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 9a89fa29858d..c25b382a78a4 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4207,7 +4207,7 @@ }, "UNSET_NONEXISTENT_PROPERTIES" : { "message" : [ - "Attempted to unset non-existent properties [] in table ." + "Attempted to unset non-existent properties [] in relation ." ], "sqlState" : "42K0J" }, diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 7501283a4ac3..8f04a40fabb2 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -105,6 +105,8 @@ statement (WITH (DBPROPERTIES | PROPERTIES) propertyList))* #createNamespace | ALTER namespace identifierReference SET (DBPROPERTIES | PROPERTIES) propertyList #setNamespaceProperties + | ALTER namespace identifierReference + UNSET (DBPROPERTIES | PROPERTIES) (IF EXISTS)? propertyList #unsetNamespaceProperties | ALTER namespace identifierReference SET locationSpec #setNamespaceLocation | DROP namespace (IF EXISTS)? identifierReference 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 0d8bfe5f988d..05c36a7a0b64 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 @@ -4662,6 +4662,26 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } } + /** + * Parse [[UnsetNamespaceProperties]] commands. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) database + * UNSET (DBPROPERTIES | PROPERTIES) [IF EXISTS] ('comment', 'key'); + * }}} + */ + override def visitUnsetNamespaceProperties( + ctx: UnsetNamespacePropertiesContext): LogicalPlan = withOrigin(ctx) { + val properties = visitPropertyKeys(ctx.propertyList) + val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq + val ifExists = ctx.EXISTS != null + UnsetNamespaceProperties( + withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), + cleanedProperties, + ifExists) + } + /** * Create an [[SetTableLocation]] command. * 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 48d48f0cd9e2..8e5470980ed9 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 @@ -633,6 +633,23 @@ case class SetNamespaceProperties( copy(namespace = newChild) } +/** + * The logical plan of the ALTER NAMESPACE UNSET PROPERTIES command. + * + * The syntax of this command is: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ...; + * }}} + */ +case class UnsetNamespaceProperties( + namespace: LogicalPlan, + propertyKeys: Seq[String], + ifExists: Boolean) extends UnaryCommand { + override def child: LogicalPlan = namespace + override protected def withNewChildInternal(newChild: LogicalPlan): UnsetNamespaceProperties = + copy(namespace = newChild) +} + /** * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command. */ 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 18a1f7dffd6b..b420f2aa7034 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 @@ -2659,12 +2659,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat } def unsetNonExistentPropertiesError( - properties: Seq[String], table: TableIdentifier): Throwable = { + properties: Seq[String], nameParts: Seq[String]): Throwable = { new AnalysisException( errorClass = "UNSET_NONEXISTENT_PROPERTIES", messageParameters = Map( "properties" -> properties.map(toSQLId).mkString(", "), - "table" -> toSQLId(table.nameParts)) + "relationId" -> toSQLId(nameParts)) ) } 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 6f402188910e..5f639bc5665a 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 @@ -340,7 +340,7 @@ case class AlterTableUnsetPropertiesCommand( && key != TableCatalog.PROP_COMMENT) if (nonexistentKeys.nonEmpty) { throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, table.identifier) + nonexistentKeys, table.identifier.nameParts) } } // If comment is in the table property, we reset it to None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala new file mode 100644 index 000000000000..76c3b3a936bd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala @@ -0,0 +1,50 @@ +/* + * 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.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * Physical plan node for unsetting properties of namespace. + */ +case class AlterNamespaceUnsetPropertiesExec( + catalog: SupportsNamespaces, + namespace: Seq[String], + propKeys: Seq[String], + ifExists: Boolean) extends LeafV2CommandExec { + override protected def run(): Seq[InternalRow] = { + if (!ifExists) { + val ns = catalog.loadNamespaceMetadata(namespace.toArray) + val nonexistentKeys = propKeys.filter(key => !ns.containsKey(key)) + if (nonexistentKeys.nonEmpty) { + throw QueryCompilationErrors.unsetNonExistentPropertiesError( + nonexistentKeys, namespace) + } + } + val changes = propKeys.map { + NamespaceChange.removeProperty + } + catalog.alterNamespace(namespace.toArray, changes: _*) + Seq.empty + } + + override def output: Seq[Attribute] = Seq.empty +} 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 7a668b75c3c7..1f386412a237 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 @@ -373,6 +373,9 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case SetNamespaceProperties(ResolvedNamespace(catalog, ns, _), properties) => AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil + case UnsetNamespaceProperties(ResolvedNamespace(catalog, ns, _), keys, ifExists) => + AlterNamespaceUnsetPropertiesExec(catalog.asNamespaceCatalog, ns, keys, ifExists) :: Nil + case SetNamespaceLocation(ResolvedNamespace(catalog, ns, _), location) => if (StringUtils.isEmpty(location)) { throw QueryExecutionErrors.invalidEmptyLocationError(location) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala new file mode 100644 index 000000000000..07635cc24593 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.plans.logical.UnsetNamespaceProperties + +class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest { + + test("unset namespace properties") { + Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken => + Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken => + comparePlans( + parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), + UnsetNamespaceProperties( + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) + + comparePlans( + parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), + UnsetNamespaceProperties( + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) + + comparePlans( + parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), + UnsetNamespaceProperties( + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a"), ifExists = false)) + } + } + } + + test("property values must not be set") { + val sql = "ALTER NAMESPACE my_db UNSET PROPERTIES('key_without_value', 'key_with_value'='x')" + checkError( + exception = parseException(parsePlan)(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/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala new file mode 100644 index 000000000000..5184b7a89556 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -0,0 +1,127 @@ +/* + * 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.parser.ParseException +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.internal.SQLConf + +/** + * This base suite contains unified tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` 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.AlterNamespaceUnsetPropertiesSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterNamespaceUnsetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceUnsetPropertiesSuite` + */ +trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER NAMESPACE ... UNSET PROPERTIES" + + protected def namespace: String + + protected def getProperties(namespace: String): String = { + val propsRow = sql(s"DESCRIBE NAMESPACE EXTENDED $namespace") + .toDF("key", "value") + .where("key like 'Properties%'") + .collect() + assert(propsRow.length == 1) + propsRow(0).getString(1) + } + + test("namespace does not exist") { + val ns = "not_exist" + val e = intercept[AnalysisException] { + sql(s"ALTER NAMESPACE $catalog.$ns UNSET PROPERTIES ('d')") + } + checkError(e, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> s"`$ns`")) + } + + test("basic test") { + val ns = s"$catalog.$namespace" + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + assert(getProperties(ns) === "") + sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('a'='a', 'b'='b', 'c'='c')") + assert(getProperties(ns) === "((a,a), (b,b), (c,c))") + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") + assert(getProperties(ns) === "((a,a), (c,c))") + + checkError( + exception = intercept[AnalysisException] { + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") + }, + errorClass = "UNSET_NONEXISTENT_PROPERTIES", + parameters = Map("properties" -> "`b`", "relationId" -> toSQLId(namespace))) + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES IF EXISTS ('b')") + } + } + + test("test reserved properties") { + import SupportsNamespaces._ + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + val ns = s"$catalog.$namespace" + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "false")) { + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespace(ns) { + sql(s"CREATE NAMESPACE $ns") + val sqlText = s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')" + checkErrorMatchPVals( + exception = intercept[ParseException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + parameters = Map( + "property" -> key, + "msg" -> ".*"), + sqlState = None, + context = ExpectedContext( + fragment = sqlText, + start = 0, + stop = 37 + ns.length + key.length) + ) + } + } + } + withSQLConf((SQLConf.LEGACY_PROPERTY_NON_RESERVED.key, "true")) { + CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => + withNamespace(ns) { + // Set the location explicitly because v2 catalog may not set the default location. + // Without this, `meta.get(key)` below may return null. + sql(s"CREATE NAMESPACE $ns LOCATION 'tmp/prop_test'") + assert(getProperties(ns) === "") + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('$key')") + assert(getProperties(ns) === "", s"$key is a reserved namespace property and ignored") + val meta = spark.sessionState.catalogManager.catalog(catalog) + .asNamespaceCatalog.loadNamespaceMetadata(namespace.split('.')) + assert(!meta.get(key).contains("foo"), + "reserved properties should not have side effects") + } + } + } + } +} 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 6bc5917edd52..d141f03cc0ea 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 @@ -347,7 +347,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`test_prop1`, `test_prop2`", - "table" -> "`spark_catalog`.`default`.`test_table`") + "relationId" -> "`spark_catalog`.`default`.`test_table`") ) } } @@ -1190,7 +1190,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") }, errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`xyz`", "table" -> "`spark_catalog`.`dbx`.`tab1`") + parameters = Map("properties" -> "`xyz`", "relationId" -> "`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')") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala new file mode 100644 index 000000000000..da7fdbba16b0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -0,0 +1,45 @@ +/* + * 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.execution.command + +/** + * This base suite contains unified tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command + * that checks 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.AlterNamespaceUnsetPropertiesSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterNamespaceUnsetPropertiesSuite` + */ +trait AlterNamespaceUnsetPropertiesSuiteBase extends command.AlterNamespaceUnsetPropertiesSuiteBase + with command.TestsV1AndV2Commands { + override def namespace: String = "db" +} + +/** + * The class contains tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command to + * check V1 In-Memory table catalog. + */ +class AlterNamespaceUnsetPropertiesSuite extends AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = + super[AlterNamespaceUnsetPropertiesSuiteBase].commandVersion +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala new file mode 100644 index 000000000000..352238eda2ea --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesSuite.scala @@ -0,0 +1,30 @@ +/* + * 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 org.apache.spark.sql.execution.command + +/** + * The class contains tests for the `ALTER NAMESPACE ... UNSET PROPERTIES` command to check V2 + * table catalogs. + */ +class AlterNamespaceUnsetPropertiesSuite extends command.AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + + override def namespace: String = "ns1.ns2" +} 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 965db22b78f1..ee117c7b3054 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 @@ -778,7 +778,7 @@ class HiveDDLSuite errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`p`", - "table" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") + "relationId" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") ) } } @@ -1896,7 +1896,7 @@ class HiveDDLSuite parameters = Map( "properties" -> (s"${(forbiddenPrefix.split("\\.") :+ "foo"). map(part => s"`$part`").mkString(".")}"), - "table" -> "`spark_catalog`.`default`.`tbl`") + "relationId" -> "`spark_catalog`.`default`.`tbl`") ) checkError( exception = intercept[AnalysisException] { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala new file mode 100644 index 000000000000..22d833649fc6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterNamespaceUnsetPropertiesSuite.scala @@ -0,0 +1,29 @@ +/* + * 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 NAMESPACE ... UNSET PROPERTIES` command to check + * V1 Hive external table catalog. + */ +class AlterNamespaceUnsetPropertiesSuite extends v1.AlterNamespaceUnsetPropertiesSuiteBase + with CommandSuiteBase { + override def commandVersion: String = super[AlterNamespaceUnsetPropertiesSuiteBase].commandVersion +} From 363ad4944abd634b31e5107ab8d54d5f4f5a6b3d Mon Sep 17 00:00:00 2001 From: panbingkun Date: Mon, 24 Jun 2024 22:22:48 +0800 Subject: [PATCH 2/9] update --- .../catalyst/analysis/ResolveCatalogs.scala | 36 ++++++++++--------- .../sql/catalyst/parser/AstBuilder.scala | 22 +----------- .../catalyst/plans/logical/v2Commands.scala | 17 --------- .../spark/sql/execution/SparkSqlParser.scala | 21 +++++++++++ .../sql/execution/command/commands.scala | 3 +- .../AlterNamespaceUnsetPropertiesExec.scala | 16 +++++---- .../datasources/v2/DataSourceV2Strategy.scala | 3 -- ...rNamespaceUnsetPropertiesParserSuite.scala | 36 +++++++++++++------ 8 files changed, 78 insertions(+), 76 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{datasources => command}/v2/AlterNamespaceUnsetPropertiesExec.scala (77%) 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 664b68008080..b6e5b115ea95 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 @@ -53,24 +53,9 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case CurrentNamespace => ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq) case UnresolvedNamespace(Seq(), fetchMetadata) => - resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) + ResolveCatalogs.resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) case UnresolvedNamespace(CatalogAndNamespace(catalog, ns), fetchMetadata) => - resolveNamespace(catalog, ns, fetchMetadata) - } - - private def resolveNamespace( - catalog: CatalogPlugin, - ns: Seq[String], - fetchMetadata: Boolean): ResolvedNamespace = { - catalog match { - case supportsNS: SupportsNamespaces if fetchMetadata => - ResolvedNamespace( - catalog, - ns, - supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) - case _ => - ResolvedNamespace(catalog, ns) - } + ResolveCatalogs.resolveNamespace(catalog, ns, fetchMetadata) } private def resolveVariableName(nameParts: Seq[String]): ResolvedIdentifier = { @@ -98,3 +83,20 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } } } + +object ResolveCatalogs { + def resolveNamespace( + catalog: CatalogPlugin, + ns: Seq[String], + fetchMetadata: Boolean): ResolvedNamespace = { + catalog match { + case supportsNS: SupportsNamespaces if fetchMetadata => + ResolvedNamespace( + catalog, + ns, + supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) + case _ => + ResolvedNamespace(catalog, ns) + } + } +} 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 698b683963d8..dc43bd163659 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 @@ -3671,7 +3671,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } } - private def cleanNamespaceProperties( + protected def cleanNamespaceProperties( properties: Map[String, String], ctx: ParserRuleContext): Map[String, String] = withOrigin(ctx) { import SupportsNamespaces._ @@ -4660,26 +4660,6 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } } - /** - * Parse [[UnsetNamespaceProperties]] commands. - * - * For example: - * {{{ - * ALTER (DATABASE|SCHEMA|NAMESPACE) database - * UNSET (DBPROPERTIES | PROPERTIES) [IF EXISTS] ('comment', 'key'); - * }}} - */ - override def visitUnsetNamespaceProperties( - ctx: UnsetNamespacePropertiesContext): LogicalPlan = withOrigin(ctx) { - val properties = visitPropertyKeys(ctx.propertyList) - val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq - val ifExists = ctx.EXISTS != null - UnsetNamespaceProperties( - withIdentClause(ctx.identifierReference, UnresolvedNamespace(_)), - cleanedProperties, - ifExists) - } - /** * Create an [[SetTableLocation]] command. * 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 8e5470980ed9..48d48f0cd9e2 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 @@ -633,23 +633,6 @@ case class SetNamespaceProperties( copy(namespace = newChild) } -/** - * The logical plan of the ALTER NAMESPACE UNSET PROPERTIES command. - * - * The syntax of this command is: - * {{{ - * ALTER (DATABASE|SCHEMA|NAMESPACE) ... UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ...; - * }}} - */ -case class UnsetNamespaceProperties( - namespace: LogicalPlan, - propertyKeys: Seq[String], - ifExists: Boolean) extends UnaryCommand { - override def child: LogicalPlan = namespace - override protected def withNewChildInternal(newChild: LogicalPlan): UnsetNamespaceProperties = - copy(namespace = newChild) -} - /** * The logical plan of the ALTER (DATABASE|SCHEMA|NAMESPACE) ... SET LOCATION command. */ 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 59169dc51415..e7fbd9b46cf1 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 @@ -37,6 +37,7 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.PARAMETER import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.command.v2.UnsetNamespacePropertiesCommand import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION @@ -1098,4 +1099,24 @@ class SparkSqlAstBuilder extends AstBuilder { (ctx.LOCAL != null, finalStorage, Some(DDLUtils.HIVE_PROVIDER)) } + + /** + * Create a [[UnsetNamespacePropertiesCommand]] command. + * + * For example: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) database + * UNSET (DBPROPERTIES | PROPERTIES) [IF EXISTS] ('comment', 'key'); + * }}} + */ + override def visitUnsetNamespaceProperties( + ctx: UnsetNamespacePropertiesContext): LogicalPlan = withOrigin(ctx) { + val properties = visitPropertyKeys(ctx.propertyList) + val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq + val ifExists = ctx.EXISTS != null + UnsetNamespacePropertiesCommand( + withIdentClause(ctx.identifierReference(), UnresolvedIdentifier(_)), + cleanedProperties, + ifExists) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index eec79ad02e29..ea2736b2c126 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan, SupervisingCommand} -import org.apache.spark.sql.catalyst.trees.LeafLike +import org.apache.spark.sql.catalyst.trees.{LeafLike, UnaryLike} import org.apache.spark.sql.connector.ExternalCommandRunner import org.apache.spark.sql.execution.{CommandExecutionMode, ExplainMode, LeafExecNode, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetric @@ -51,6 +51,7 @@ trait RunnableCommand extends Command { } trait LeafRunnableCommand extends RunnableCommand with LeafLike[LogicalPlan] +trait UnaryRunnableCommand extends RunnableCommand with UnaryLike[LogicalPlan] /** * A physical operator that executes the run method of a `RunnableCommand` and diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala similarity index 77% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala index 76c3b3a936bd..d52d6d52d7b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterNamespaceUnsetPropertiesExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala @@ -15,34 +15,38 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.v2 +package org.apache.spark.sql.execution.command.v2 import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec /** * Physical plan node for unsetting properties of namespace. */ case class AlterNamespaceUnsetPropertiesExec( catalog: SupportsNamespaces, - namespace: Seq[String], + namespace1: Seq[String], + namespace: ResolvedNamespace, propKeys: Seq[String], ifExists: Boolean) extends LeafV2CommandExec { + override protected def run(): Seq[InternalRow] = { if (!ifExists) { - val ns = catalog.loadNamespaceMetadata(namespace.toArray) - val nonexistentKeys = propKeys.filter(key => !ns.containsKey(key)) + val properties = namespace.metadata + val nonexistentKeys = propKeys.filter(key => !properties.contains(key)) if (nonexistentKeys.nonEmpty) { throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, namespace) + nonexistentKeys, namespace.namespace) } } val changes = propKeys.map { NamespaceChange.removeProperty } - catalog.alterNamespace(namespace.toArray, changes: _*) + // catalog.alterNamespace(namespace.toArray, changes: _*) Seq.empty } 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 1f386412a237..7a668b75c3c7 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 @@ -373,9 +373,6 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case SetNamespaceProperties(ResolvedNamespace(catalog, ns, _), properties) => AlterNamespaceSetPropertiesExec(catalog.asNamespaceCatalog, ns, properties) :: Nil - case UnsetNamespaceProperties(ResolvedNamespace(catalog, ns, _), keys, ifExists) => - AlterNamespaceUnsetPropertiesExec(catalog.asNamespaceCatalog, ns, keys, ifExists) :: Nil - case SetNamespaceLocation(ResolvedNamespace(catalog, ns, _), location) => if (StringUtils.isEmpty(location)) { throw QueryExecutionErrors.invalidEmptyLocationError(location) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index 07635cc24593..f37e63f179d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -16,29 +16,43 @@ */ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} -import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan -import org.apache.spark.sql.catalyst.plans.logical.UnsetNamespaceProperties +import org.apache.spark.SparkThrowable +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkSqlParser +import org.apache.spark.sql.execution.command.v2.UnsetNamespacePropertiesCommand +import org.apache.spark.sql.test.SharedSparkSession -class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest { +class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedSparkSession { + + private lazy val parser = new SparkSqlParser() + + private def parseException(sqlText: String): SparkThrowable = { + intercept[ParseException](sql(sqlText).collect()) + } + + private def parsePlan(sqlText: String): LogicalPlan = { + parser.parsePlan(sqlText) + } test("unset namespace properties") { Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken => Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken => comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), - UnsetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) + UnsetNamespacePropertiesCommand( + UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), - UnsetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) + UnsetNamespacePropertiesCommand( + UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), - UnsetNamespaceProperties( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a"), ifExists = false)) + UnsetNamespacePropertiesCommand( + UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a"), ifExists = false)) } } } @@ -46,7 +60,7 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest { test("property values must not be set") { val sql = "ALTER NAMESPACE my_db UNSET PROPERTIES('key_without_value', 'key_with_value'='x')" checkError( - exception = parseException(parsePlan)(sql), + exception = parseException(sql), errorClass = "_LEGACY_ERROR_TEMP_0035", parameters = Map("message" -> "Values should not be specified for key(s): [key_with_value]"), context = ExpectedContext( From 1f3ed06bd8ef967bf8e852dcd7cc9ddb4219b6c4 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 25 Jun 2024 09:25:46 +0800 Subject: [PATCH 3/9] fix --- .../spark/sql/execution/SparkSqlParser.scala | 1 - .../AlterNamespaceUnsetPropertiesExec.scala | 54 ------------------- ...rNamespaceUnsetPropertiesParserSuite.scala | 1 - 3 files changed, 56 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala 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 e7fbd9b46cf1..04314f5d722f 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 @@ -37,7 +37,6 @@ import org.apache.spark.sql.catalyst.trees.TreePattern.PARAMETER import org.apache.spark.sql.catalyst.util.DateTimeConstants import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors} import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.command.v2.UnsetNamespacePropertiesCommand import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf, VariableSubstitution} import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala deleted file mode 100644 index d52d6d52d7b4..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/v2/AlterNamespaceUnsetPropertiesExec.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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 org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} -import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.datasources.v2.LeafV2CommandExec - -/** - * Physical plan node for unsetting properties of namespace. - */ -case class AlterNamespaceUnsetPropertiesExec( - catalog: SupportsNamespaces, - namespace1: Seq[String], - namespace: ResolvedNamespace, - propKeys: Seq[String], - ifExists: Boolean) extends LeafV2CommandExec { - - override protected def run(): Seq[InternalRow] = { - if (!ifExists) { - val properties = namespace.metadata - val nonexistentKeys = propKeys.filter(key => !properties.contains(key)) - if (nonexistentKeys.nonEmpty) { - throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, namespace.namespace) - } - } - val changes = propKeys.map { - NamespaceChange.removeProperty - } - // catalog.alterNamespace(namespace.toArray, changes: _*) - Seq.empty - } - - override def output: Seq[Attribute] = Seq.empty -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index f37e63f179d5..4c4495b087ee 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifie import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkSqlParser -import org.apache.spark.sql.execution.command.v2.UnsetNamespacePropertiesCommand import org.apache.spark.sql.test.SharedSparkSession class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedSparkSession { From 960f32a110704028981bcdcf0e09546c926e6d1e Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 25 Jun 2024 09:57:38 +0800 Subject: [PATCH 4/9] update --- .../UnsetNamespacePropertiesCommand.scala | 70 +++++++++++++++++++ 1 file changed, 70 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala new file mode 100644 index 000000000000..6f58f1600eb4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala @@ -0,0 +1,70 @@ +/* + * 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.{Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.{ResolveCatalogs, ResolvedIdentifier} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * A command that ALTER NAMESPACE UNSET PROPERTIES command. + * + * The syntax of this command is: + * {{{ + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ...; + * }}} + */ +case class UnsetNamespacePropertiesCommand( + ident: LogicalPlan, + propKeys: Seq[String], + ifExists: Boolean) extends UnaryRunnableCommand { + + override def run(sparkSession: SparkSession): Seq[Row] = { + val ResolvedIdentifier(catalog, ident) = child + val ns = ResolveCatalogs.resolveNamespace( + catalog, ident.namespace.toSeq :+ ident.name, fetchMetadata = true) + if (!ifExists) { + val properties = ns.metadata + val nonexistentKeys = propKeys.filter(key => !properties.contains(key)) + if (nonexistentKeys.nonEmpty) { + throw QueryCompilationErrors.unsetNonExistentPropertiesError( + nonexistentKeys, ns.namespace) + } + } + val changes = propKeys.map { + NamespaceChange.removeProperty + } + catalog match { + case supportsNS: SupportsNamespaces => + supportsNS.alterNamespace(ns.namespace.toArray, changes: _*) + case _ => + QueryCompilationErrors.catalogOperationNotSupported( + catalog, "UNSET (DBPROPERTIES|PROPERTIES)") + } + + Seq.empty + } + + override def child: LogicalPlan = ident + + override protected def withNewChildInternal( + newChild: LogicalPlan): UnsetNamespacePropertiesCommand = + copy(ident = newChild) +} From 53a9a25f20d2fab2ee14331325ce055beaa3e182 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 25 Jun 2024 15:46:35 +0800 Subject: [PATCH 5/9] update --- .../resources/error/error-conditions.json | 12 +++---- .../spark/sql/errors/QueryParsingErrors.scala | 2 +- .../catalyst/analysis/ResolveCatalogs.scala | 36 +++++++++---------- .../sql/errors/QueryCompilationErrors.scala | 2 +- .../spark/sql/execution/SparkSqlParser.scala | 4 +-- .../UnsetNamespacePropertiesCommand.scala | 22 ++++-------- .../sql/errors/QueryParsingErrorsSuite.scala | 2 +- ...AlterNamespaceSetPropertiesSuiteBase.scala | 17 ++++++--- ...rNamespaceUnsetPropertiesParserSuite.scala | 8 ++--- ...terNamespaceUnsetPropertiesSuiteBase.scala | 8 ++--- .../sql/execution/command/DDLSuite.scala | 4 +-- .../sql/hive/execution/HiveDDLSuite.scala | 4 +-- 12 files changed, 59 insertions(+), 62 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 4d7bfc405379..6862415cd93f 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4231,7 +4231,7 @@ }, "UNSET_NONEXISTENT_PROPERTIES" : { "message" : [ - "Attempted to unset non-existent properties [] in relation ." + "Attempted to unset non-existent properties [] in table or namespace ." ], "sqlState" : "42K0J" }, @@ -4411,6 +4411,11 @@ " with AES- does not support initialization vectors (IVs)." ] }, + "ALTER_NAMESPACE_PROPERTY" : { + "message" : [ + " is a reserved namespace property, ." + ] + }, "ANALYZE_UNCACHED_TEMP_VIEW" : { "message" : [ "The ANALYZE TABLE FOR COLUMNS command can operate on temporary views that have been cached already. Consider to cache the view ." @@ -4576,11 +4581,6 @@ "The replace function does not support nested column ." ] }, - "SET_NAMESPACE_PROPERTY" : { - "message" : [ - " is a reserved namespace property, ." - ] - }, "SET_OPERATION_ON_MAP_TYPE" : { "message" : [ "Cannot have MAP type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column is ." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index e7ae9f2bfb7b..a8c3e722c376 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -339,7 +339,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { def cannotCleanReservedNamespacePropertyError( property: String, ctx: ParserRuleContext, msg: String): Throwable = { new ParseException( - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", messageParameters = Map("property" -> property, "msg" -> msg), ctx) } 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 b6e5b115ea95..664b68008080 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 @@ -53,9 +53,24 @@ class ResolveCatalogs(val catalogManager: CatalogManager) case CurrentNamespace => ResolvedNamespace(currentCatalog, catalogManager.currentNamespace.toImmutableArraySeq) case UnresolvedNamespace(Seq(), fetchMetadata) => - ResolveCatalogs.resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) + resolveNamespace(currentCatalog, Seq.empty[String], fetchMetadata) case UnresolvedNamespace(CatalogAndNamespace(catalog, ns), fetchMetadata) => - ResolveCatalogs.resolveNamespace(catalog, ns, fetchMetadata) + resolveNamespace(catalog, ns, fetchMetadata) + } + + private def resolveNamespace( + catalog: CatalogPlugin, + ns: Seq[String], + fetchMetadata: Boolean): ResolvedNamespace = { + catalog match { + case supportsNS: SupportsNamespaces if fetchMetadata => + ResolvedNamespace( + catalog, + ns, + supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) + case _ => + ResolvedNamespace(catalog, ns) + } } private def resolveVariableName(nameParts: Seq[String]): ResolvedIdentifier = { @@ -83,20 +98,3 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } } } - -object ResolveCatalogs { - def resolveNamespace( - catalog: CatalogPlugin, - ns: Seq[String], - fetchMetadata: Boolean): ResolvedNamespace = { - catalog match { - case supportsNS: SupportsNamespaces if fetchMetadata => - ResolvedNamespace( - catalog, - ns, - supportsNS.loadNamespaceMetadata(ns.toArray).asScala.toMap) - case _ => - ResolvedNamespace(catalog, ns) - } - } -} 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 b9bb9ad5e180..20f70570b7f5 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 @@ -2664,7 +2664,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat errorClass = "UNSET_NONEXISTENT_PROPERTIES", messageParameters = Map( "properties" -> properties.map(toSQLId).mkString(", "), - "relationId" -> toSQLId(nameParts)) + "name" -> toSQLId(nameParts)) ) } 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 04314f5d722f..96ee173c307a 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 @@ -27,7 +27,7 @@ import org.antlr.v4.runtime.tree.TerminalNode import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, SchemaEvolution, SchemaTypeEvolution, UnresolvedFunctionName, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, PersistedView, SchemaEvolution, SchemaTypeEvolution, UnresolvedFunctionName, UnresolvedIdentifier, UnresolvedNamespace} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} import org.apache.spark.sql.catalyst.parser._ @@ -1114,7 +1114,7 @@ class SparkSqlAstBuilder extends AstBuilder { val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq val ifExists = ctx.EXISTS != null UnsetNamespacePropertiesCommand( - withIdentClause(ctx.identifierReference(), UnresolvedIdentifier(_)), + withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_, fetchMetadata = true)), cleanedProperties, ifExists) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala index 6f58f1600eb4..54c77e78bc43 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.analysis.{ResolveCatalogs, ResolvedIdentifier} +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +import org.apache.spark.sql.connector.catalog.NamespaceChange import org.apache.spark.sql.errors.QueryCompilationErrors /** @@ -37,27 +38,18 @@ case class UnsetNamespacePropertiesCommand( ifExists: Boolean) extends UnaryRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - val ResolvedIdentifier(catalog, ident) = child - val ns = ResolveCatalogs.resolveNamespace( - catalog, ident.namespace.toSeq :+ ident.name, fetchMetadata = true) + val ResolvedNamespace(catalog, ns, metadata) = child if (!ifExists) { - val properties = ns.metadata - val nonexistentKeys = propKeys.filter(key => !properties.contains(key)) + val nonexistentKeys = propKeys.filter(key => !metadata.contains(key)) if (nonexistentKeys.nonEmpty) { throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, ns.namespace) + nonexistentKeys, ns) } } val changes = propKeys.map { NamespaceChange.removeProperty } - catalog match { - case supportsNS: SupportsNamespaces => - supportsNS.alterNamespace(ns.namespace.toArray, changes: _*) - case _ => - QueryCompilationErrors.catalogOperationNotSupported( - catalog, "UNSET (DBPROPERTIES|PROPERTIES)") - } + catalog.asNamespaceCatalog.alterNamespace(ns.toArray, changes: _*) Seq.empty } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index b7fb65091ef7..1a97f66f4650 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -562,7 +562,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "CREATE NAMESPACE IF NOT EXISTS a.b.c WITH PROPERTIES ('location'='/home/user/db')" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", sqlState = "0A000", parameters = Map( "property" -> "location", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index c28c7b9db043..4c4f18793801 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -83,10 +83,19 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filterNot(_ == PROP_COMMENT).foreach { key => withNamespace(ns) { sql(s"CREATE NAMESPACE $ns") - val exception = intercept[ParseException] { - sql(s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='dummyVal')") - } - assert(exception.getMessage.contains(s"$key is a reserved namespace property")) + val sqlText = s"ALTER NAMESPACE $ns SET PROPERTIES ('$key'='dummyVal')" + checkErrorMatchPVals( + exception = intercept[ParseException] { + sql(sqlText) + }, + errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + parameters = Map("property" -> key, "msg" -> ".*"), + sqlState = None, + context = ExpectedContext( + fragment = sqlText, + start = 0, + stop = 46 + ns.length + key.length) + ) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index 4c4495b087ee..3f680494d42b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.SparkThrowable -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedNamespace} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkSqlParser @@ -41,17 +41,17 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), UnsetNamespacePropertiesCommand( - UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), UnsetNamespacePropertiesCommand( - UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), UnsetNamespacePropertiesCommand( - UnresolvedIdentifier(Seq("a", "b", "c")), Seq("a"), ifExists = false)) + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a"), ifExists = false)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala index 5184b7a89556..2d41d202240c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -76,7 +76,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") }, errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`b`", "relationId" -> toSQLId(namespace))) + parameters = Map("properties" -> "`b`", "name" -> toSQLId(namespace))) sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES IF EXISTS ('b')") } } @@ -94,10 +94,8 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", - parameters = Map( - "property" -> key, - "msg" -> ".*"), + errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + parameters = Map("property" -> key, "msg" -> ".*"), sqlState = None, context = ExpectedContext( fragment = sqlText, 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 955e002d45b1..5d93bfc75862 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 @@ -343,7 +343,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`test_prop1`, `test_prop2`", - "relationId" -> "`spark_catalog`.`default`.`test_table`") + "name" -> "`spark_catalog`.`default`.`test_table`") ) } } @@ -1152,7 +1152,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") }, errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`xyz`", "relationId" -> "`spark_catalog`.`dbx`.`tab1`") + parameters = Map("properties" -> "`xyz`", "name" -> "`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')") 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 0d51c3aec0db..0146078771d8 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 @@ -774,7 +774,7 @@ class HiveDDLSuite errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`p`", - "relationId" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") + "name" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") ) } } @@ -1892,7 +1892,7 @@ class HiveDDLSuite parameters = Map( "properties" -> (s"${(forbiddenPrefix.split("\\.") :+ "foo"). map(part => s"`$part`").mkString(".")}"), - "relationId" -> "`spark_catalog`.`default`.`tbl`") + "name" -> "`spark_catalog`.`default`.`tbl`") ) checkError( exception = intercept[AnalysisException] { From 33fbc7c3e52249de38a7f6959f8ef283e7cd04fe Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 25 Jun 2024 17:25:44 +0800 Subject: [PATCH 6/9] update --- .../src/main/resources/error/error-conditions.json | 10 +++++----- .../apache/spark/sql/errors/QueryParsingErrors.scala | 2 +- .../command/UnsetNamespacePropertiesCommand.scala | 5 +++-- .../spark/sql/errors/QueryParsingErrorsSuite.scala | 2 +- .../command/AlterNamespaceSetPropertiesSuiteBase.scala | 2 +- .../AlterNamespaceUnsetPropertiesSuiteBase.scala | 3 ++- 6 files changed, 13 insertions(+), 11 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 6862415cd93f..8d8a8c4245a2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4411,11 +4411,6 @@ " with AES- does not support initialization vectors (IVs)." ] }, - "ALTER_NAMESPACE_PROPERTY" : { - "message" : [ - " is a reserved namespace property, ." - ] - }, "ANALYZE_UNCACHED_TEMP_VIEW" : { "message" : [ "The ANALYZE TABLE FOR COLUMNS command can operate on temporary views that have been cached already. Consider to cache the view ." @@ -4581,6 +4576,11 @@ "The replace function does not support nested column ." ] }, + "SET_NAMESPACE_PROPERTY" : { + "message" : [ + " is a reserved namespace property, ." + ] + }, "SET_OPERATION_ON_MAP_TYPE" : { "message" : [ "Cannot have MAP type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column is ." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index a8c3e722c376..e7ae9f2bfb7b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -339,7 +339,7 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { def cannotCleanReservedNamespacePropertyError( property: String, ctx: ParserRuleContext, msg: String): Throwable = { new ParseException( - errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", messageParameters = Map("property" -> property, "msg" -> msg), ctx) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala index 54c77e78bc43..2b0c82a0fe67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala @@ -25,11 +25,12 @@ import org.apache.spark.sql.connector.catalog.NamespaceChange import org.apache.spark.sql.errors.QueryCompilationErrors /** - * A command that ALTER NAMESPACE UNSET PROPERTIES command. + * A command that unsets database/schema/namespace properties. * * The syntax of this command is: * {{{ - * ALTER (DATABASE|SCHEMA|NAMESPACE) ... UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ...; + * ALTER (DATABASE|SCHEMA|NAMESPACE) ... + * UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ('key1', 'key2', ...); * }}} */ case class UnsetNamespacePropertiesCommand( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 1a97f66f4650..b7fb65091ef7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -562,7 +562,7 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL val sqlText = "CREATE NAMESPACE IF NOT EXISTS a.b.c WITH PROPERTIES ('location'='/home/user/db')" checkError( exception = parseException(sqlText), - errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", sqlState = "0A000", parameters = Map( "property" -> "location", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index 4c4f18793801..7f5b3de4865c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -88,7 +88,7 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", parameters = Map("property" -> key, "msg" -> ".*"), sqlState = None, context = ExpectedContext( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala index 2d41d202240c..a50ab8bcdbb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -71,6 +71,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") assert(getProperties(ns) === "((a,a), (c,c))") + // unset non-existent properties checkError( exception = intercept[AnalysisException] { sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") @@ -94,7 +95,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe exception = intercept[ParseException] { sql(sqlText) }, - errorClass = "UNSUPPORTED_FEATURE.ALTER_NAMESPACE_PROPERTY", + errorClass = "UNSUPPORTED_FEATURE.SET_NAMESPACE_PROPERTY", parameters = Map("property" -> key, "msg" -> ".*"), sqlState = None, context = ExpectedContext( From 0d09a651906358c11c458e9846edeabc6cf97a80 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Tue, 25 Jun 2024 21:22:41 +0800 Subject: [PATCH 7/9] fix AlterNamespaceUnsetPropertiesParserSuite --- .../AlterNamespaceUnsetPropertiesParserSuite.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index 3f680494d42b..244620c3feb0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -41,17 +41,20 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = false)) + UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), + Seq("a", "b", "c"), ifExists = false)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"), ifExists = true)) + UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), + Seq("a", "b", "c"), ifExists = true)) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c")), Seq("a"), ifExists = false)) + UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), + Seq("a"), ifExists = false)) } } } From 0e925c67c516c108bd71a53c36f1871c0f93a0b3 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 27 Jun 2024 09:40:23 +0800 Subject: [PATCH 8/9] remove IF EXISTS --- .../resources/error/error-conditions.json | 2 +- docs/sql-ref-syntax-ddl-alter-database.md | 19 +++++++++++++- .../sql/catalyst/parser/SqlBaseParser.g4 | 2 +- .../sql/errors/QueryCompilationErrors.scala | 4 +-- .../spark/sql/execution/SparkSqlParser.scala | 8 +++--- .../UnsetNamespacePropertiesCommand.scala | 16 +++--------- .../spark/sql/execution/command/ddl.scala | 2 +- ...rNamespaceUnsetPropertiesParserSuite.scala | 26 +++++++++++-------- ...terNamespaceUnsetPropertiesSuiteBase.scala | 11 +++----- .../sql/execution/command/DDLSuite.scala | 4 +-- .../sql/hive/execution/HiveDDLSuite.scala | 4 +-- 11 files changed, 52 insertions(+), 46 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 15dd761b5da5..2d9ae7a89b81 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4237,7 +4237,7 @@ }, "UNSET_NONEXISTENT_PROPERTIES" : { "message" : [ - "Attempted to unset non-existent properties [] in table or namespace ." + "Attempted to unset non-existent properties [] in table
." ], "sqlState" : "42K0J" }, diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md index 0ac003823643..ed720eb4c722 100644 --- a/docs/sql-ref-syntax-ddl-alter-database.md +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -25,7 +25,7 @@ license: | `DATABASE`, `SCHEMA` and `NAMESPACE` are interchangeable and one can be used in place of the others. An error message is issued if the database is not found in the system. -### ALTER PROPERTIES +### ALTER SET PROPERTIES `ALTER DATABASE SET DBPROPERTIES` statement changes the properties associated with a database. The specified property values override any existing value with the same property name. This command is mostly used to record the metadata for a database and may be used for auditing purposes. @@ -43,6 +43,23 @@ ALTER { DATABASE | SCHEMA | NAMESPACE } database_name Specifies the name of the database to be altered. +### ALTER UNSET PROPERTIES +`ALTER DATABASE UNSET DBPROPERTIES` statement unsets the properties associated with a database. +If the specified property key does not exist, the command will ignore it and finally succeed. + +#### Syntax + +```sql +ALTER { DATABASE | SCHEMA | NAMESPACE } database_name + UNSET { DBPROPERTIES | PROPERTIES } ( property_name [ , ... ] ) +``` + +#### Parameters + +* **database_name** + + Specifies the name of the database to be altered. + ### ALTER LOCATION `ALTER DATABASE SET LOCATION` statement changes the default parent-directory where new tables will be added for a database. Please note that it does not move the contents of the database's current directory to the newly diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 3b3e2202e9e2..54eff14b6d4d 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -106,7 +106,7 @@ statement | ALTER namespace identifierReference SET (DBPROPERTIES | PROPERTIES) propertyList #setNamespaceProperties | ALTER namespace identifierReference - UNSET (DBPROPERTIES | PROPERTIES) (IF EXISTS)? propertyList #unsetNamespaceProperties + UNSET (DBPROPERTIES | PROPERTIES) propertyList #unsetNamespaceProperties | ALTER namespace identifierReference SET locationSpec #setNamespaceLocation | DROP namespace (IF EXISTS)? identifierReference 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 20f70570b7f5..d3bd265d0459 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 @@ -2659,12 +2659,12 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat } def unsetNonExistentPropertiesError( - properties: Seq[String], nameParts: Seq[String]): Throwable = { + properties: Seq[String], table: TableIdentifier): Throwable = { new AnalysisException( errorClass = "UNSET_NONEXISTENT_PROPERTIES", messageParameters = Map( "properties" -> properties.map(toSQLId).mkString(", "), - "name" -> toSQLId(nameParts)) + "table" -> toSQLId(table.nameParts)) ) } 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 96ee173c307a..ea36dafe6739 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 @@ -1105,17 +1105,15 @@ class SparkSqlAstBuilder extends AstBuilder { * For example: * {{{ * ALTER (DATABASE|SCHEMA|NAMESPACE) database - * UNSET (DBPROPERTIES | PROPERTIES) [IF EXISTS] ('comment', 'key'); + * UNSET (DBPROPERTIES | PROPERTIES) ('comment', 'key'); * }}} */ override def visitUnsetNamespaceProperties( ctx: UnsetNamespacePropertiesContext): LogicalPlan = withOrigin(ctx) { val properties = visitPropertyKeys(ctx.propertyList) val cleanedProperties = cleanNamespaceProperties(properties.map(_ -> "").toMap, ctx).keys.toSeq - val ifExists = ctx.EXISTS != null UnsetNamespacePropertiesCommand( - withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_, fetchMetadata = true)), - cleanedProperties, - ifExists) + withIdentClause(ctx.identifierReference(), UnresolvedNamespace(_)), + cleanedProperties) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala index 2b0c82a0fe67..243b51b09e3b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/UnsetNamespacePropertiesCommand.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper import org.apache.spark.sql.connector.catalog.NamespaceChange -import org.apache.spark.sql.errors.QueryCompilationErrors /** * A command that unsets database/schema/namespace properties. @@ -30,26 +29,19 @@ import org.apache.spark.sql.errors.QueryCompilationErrors * The syntax of this command is: * {{{ * ALTER (DATABASE|SCHEMA|NAMESPACE) ... - * UNSET (DBPROPERTIES|PROPERTIES) [IF EXISTS] ('key1', 'key2', ...); + * UNSET (DBPROPERTIES|PROPERTIES) ('key1', 'key2', ...); * }}} */ case class UnsetNamespacePropertiesCommand( ident: LogicalPlan, - propKeys: Seq[String], - ifExists: Boolean) extends UnaryRunnableCommand { + propKeys: Seq[String]) extends UnaryRunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - val ResolvedNamespace(catalog, ns, metadata) = child - if (!ifExists) { - val nonexistentKeys = propKeys.filter(key => !metadata.contains(key)) - if (nonexistentKeys.nonEmpty) { - throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, ns) - } - } + val ResolvedNamespace(catalog, ns, _) = child val changes = propKeys.map { NamespaceChange.removeProperty } + // If the property does not exist, the change should succeed. catalog.asNamespaceCatalog.alterNamespace(ns.toArray, changes: _*) Seq.empty 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 5f639bc5665a..6f402188910e 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 @@ -340,7 +340,7 @@ case class AlterTableUnsetPropertiesCommand( && key != TableCatalog.PROP_COMMENT) if (nonexistentKeys.nonEmpty) { throw QueryCompilationErrors.unsetNonExistentPropertiesError( - nonexistentKeys, table.identifier.nameParts) + nonexistentKeys, table.identifier) } } // If comment is in the table property, we reset it to None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala index 244620c3feb0..72d307c81666 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesParserSuite.scala @@ -41,20 +41,11 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a', 'b', 'c')"), UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), - Seq("a", "b", "c"), ifExists = false)) - - comparePlans( - parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')"), - UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), - Seq("a", "b", "c"), ifExists = true)) + UnresolvedNamespace(Seq("a", "b", "c")), Seq("a", "b", "c"))) comparePlans( parsePlan(s"ALTER $nsToken a.b.c UNSET $propToken ('a')"), - UnsetNamespacePropertiesCommand( - UnresolvedNamespace(Seq("a", "b", "c"), fetchMetadata = true), - Seq("a"), ifExists = false)) + UnsetNamespacePropertiesCommand(UnresolvedNamespace(Seq("a", "b", "c")), Seq("a"))) } } } @@ -70,4 +61,17 @@ class AlterNamespaceUnsetPropertiesParserSuite extends AnalysisTest with SharedS start = 0, stop = 80)) } + + test("not support clause - IF EXISTS") { + Seq("DATABASE", "SCHEMA", "NAMESPACE").foreach { nsToken => + Seq("PROPERTIES", "DBPROPERTIES").foreach { propToken => + val sql = s"ALTER $nsToken a.b.c UNSET $propToken IF EXISTS ('a', 'b', 'c')" + checkError( + exception = parseException(sql), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'IF'", "hint" -> ": missing '('") + ) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala index a50ab8bcdbb1..1d43cc593848 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces} -import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.internal.SQLConf /** @@ -72,13 +71,9 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe assert(getProperties(ns) === "((a,a), (c,c))") // unset non-existent properties - checkError( - exception = intercept[AnalysisException] { - sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") - }, - errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`b`", "name" -> toSQLId(namespace))) - sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES IF EXISTS ('b')") + // it will be successful, ignoring non-existent properties + sql(s"ALTER NAMESPACE $ns UNSET PROPERTIES ('b')") + assert(getProperties(ns) === "((a,a), (c,c))") } } 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 5d93bfc75862..553b68bec52f 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 @@ -343,7 +343,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`test_prop1`, `test_prop2`", - "name" -> "`spark_catalog`.`default`.`test_table`") + "table" -> "`spark_catalog`.`default`.`test_table`") ) } } @@ -1152,7 +1152,7 @@ abstract class DDLSuite extends QueryTest with DDLSuiteBase { sql("ALTER TABLE tab1 UNSET TBLPROPERTIES ('c', 'xyz')") }, errorClass = "UNSET_NONEXISTENT_PROPERTIES", - parameters = Map("properties" -> "`xyz`", "name" -> "`spark_catalog`.`dbx`.`tab1`") + 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')") 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 0146078771d8..65b70ad8bcae 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 @@ -774,7 +774,7 @@ class HiveDDLSuite errorClass = "UNSET_NONEXISTENT_PROPERTIES", parameters = Map( "properties" -> "`p`", - "name" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") + "table" -> s"`$SESSION_CATALOG_NAME`.`default`.`view1`") ) } } @@ -1892,7 +1892,7 @@ class HiveDDLSuite parameters = Map( "properties" -> (s"${(forbiddenPrefix.split("\\.") :+ "foo"). map(part => s"`$part`").mkString(".")}"), - "name" -> "`spark_catalog`.`default`.`tbl`") + "table" -> "`spark_catalog`.`default`.`tbl`") ) checkError( exception = intercept[AnalysisException] { From f5efc77ae04349f8ca9794272f227d75f07d9255 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 27 Jun 2024 16:26:56 +0800 Subject: [PATCH 9/9] update docs --- docs/sql-ref-syntax-ddl-alter-database.md | 25 ++++++++++++++++--- .../spark/sql/execution/SparkSqlParser.scala | 4 +-- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/docs/sql-ref-syntax-ddl-alter-database.md b/docs/sql-ref-syntax-ddl-alter-database.md index ed720eb4c722..727fedb19e23 100644 --- a/docs/sql-ref-syntax-ddl-alter-database.md +++ b/docs/sql-ref-syntax-ddl-alter-database.md @@ -25,7 +25,7 @@ license: | `DATABASE`, `SCHEMA` and `NAMESPACE` are interchangeable and one can be used in place of the others. An error message is issued if the database is not found in the system. -### ALTER SET PROPERTIES +### SET PROPERTIES `ALTER DATABASE SET DBPROPERTIES` statement changes the properties associated with a database. The specified property values override any existing value with the same property name. This command is mostly used to record the metadata for a database and may be used for auditing purposes. @@ -43,9 +43,10 @@ ALTER { DATABASE | SCHEMA | NAMESPACE } database_name Specifies the name of the database to be altered. -### ALTER UNSET PROPERTIES +### UNSET PROPERTIES `ALTER DATABASE UNSET DBPROPERTIES` statement unsets the properties associated with a database. If the specified property key does not exist, the command will ignore it and finally succeed. +(available since Spark 4.0.0). #### Syntax @@ -60,7 +61,7 @@ ALTER { DATABASE | SCHEMA | NAMESPACE } database_name Specifies the name of the database to be altered. -### ALTER LOCATION +### SET LOCATION `ALTER DATABASE SET LOCATION` statement changes the default parent-directory where new tables will be added for a database. Please note that it does not move the contents of the database's current directory to the newly specified location or change the locations associated with any tables/partitions under the specified database @@ -112,6 +113,24 @@ DESCRIBE DATABASE EXTENDED inventory; | Location|file:/temp/spark-warehouse/new_inventory.db| | Properties| ((Edit-date,01/01/2001), (Edited-by,John))| +-------------------------+-------------------------------------------+ + +-- Alters the database to unset the property `Edited-by` +ALTER DATABASE inventory UNSET DBPROPERTIES ('Edited-by'); + +-- Verify that the property `Edited-by` has been unset. +DESCRIBE DATABASE EXTENDED inventory; ++-------------------------+-------------------------------------------+ +|database_description_item| database_description_value| ++-------------------------+-------------------------------------------+ +| Database Name| inventory| +| Description| | +| Location|file:/temp/spark-warehouse/new_inventory.db| +| Properties| ((Edit-date,01/01/2001)) | ++-------------------------+-------------------------------------------+ + +-- Alters the database to unset a non-existent property `non-existent` +-- Note: The command will ignore 'non-existent' and finally succeed +ALTER DATABASE inventory UNSET DBPROPERTIES ('non-existent'); ``` ### Related Statements 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 ea36dafe6739..055fec02d2ae 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 @@ -1102,10 +1102,10 @@ class SparkSqlAstBuilder extends AstBuilder { /** * Create a [[UnsetNamespacePropertiesCommand]] command. * - * For example: + * Expected format: * {{{ * ALTER (DATABASE|SCHEMA|NAMESPACE) database - * UNSET (DBPROPERTIES | PROPERTIES) ('comment', 'key'); + * UNSET (DBPROPERTIES | PROPERTIES) ('key1', 'key2'); * }}} */ override def visitUnsetNamespaceProperties(