You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/02/05 20:44:05 UTC

[GitHub] [spark] imback82 opened a new pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

imback82 opened a new pull request #31494:
URL: https://github.com/apache/spark/pull/31494


   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   This PR proposes to support `ifExists` flag for v2 `ALTER TABLE ... UNSET TBLPROPERTIES` command. Currently, the flag is not respected and the command behaves as `ifExists = true` where the command always succeeds when the properties do not exist.
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   To support `ifExists` flag and align with v1 command behavior.
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   Yes, now if the property does not exist and `IF EXISTS` is not specified, the command will fail:
   ```
   ALTER TABLE t UNSET TBLPROPERTIES ('unknown') // Fails with "Attempted to unset non-existent property 'unknown'"
   ALTER TABLE t UNSET TBLPROPERTIES IF EXISTS ('unknown') // OK
   ```
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   Added new test


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774336633


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39527/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r574240206



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {

Review comment:
       thanks, updated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774379652


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134941/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581289955



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Good catch. I don't have a strong opinion here; both approaches sound reasonable to me, each with pros/cons.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775526942


   **[Test build #135043 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135043/testReport)** for PR 31494 at commit [`8b5508b`](https://github.com/apache/spark/commit/8b5508b2625f443362d17b377f6563fdfb897b77).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774388999


   **[Test build #134945 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134945/testReport)** for PR 31494 at commit [`dbbde92`](https://github.com/apache/spark/commit/dbbde92612ff4084905465174d027f8f13100679).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572333847



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {
+    val t = s"${catalogAndNamespace}table_name"
+    withTable(t) {
+      sql(s"CREATE TABLE $t (id int) USING $v2Format TBLPROPERTIES('test' = '34')")
+
+      val tableName = fullTableName(t)
+      val table = getTableMetadata(tableName)
+
+      assert(table.name === tableName)
+      assert(table.properties ===
+        withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava)
+
+      val exc = intercept[AnalysisException] {
+        sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('unknown')")
+      }
+      assert(exc.getMessage.contains("Attempted to unset non-existent property 'unknown'"))
+
+      // Reserved property "comment" should be allowed regardless.

Review comment:
       even if the table has no comment?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777251620


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39692/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774357618


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39528/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581567000



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       @HyukjinKwon then it will be a new requirement for the v2 implementations and is a breaking change. It also means we push more work to the v2 implementation side and is not good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571342363



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -437,9 +437,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       }.toSeq
       AlterTableExec(table.catalog, table.identifier, changes) :: Nil
 
-    // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag.
-    case AlterTableUnsetProperties(table: ResolvedTable, keys, _) =>
-      val changes = keys.map(key => TableChange.removeProperty(key))
+    case AlterTableUnsetProperties(table: ResolvedTable, keys, ifExists) =>
+      val changes = keys.map(key => TableChange.removeProperty(key, ifExists))

Review comment:
       Another approach is to do the existence check here instead of changing the public APIs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571282171



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala
##########
@@ -116,7 +116,11 @@ private[sql] object CatalogV2Util {
         newProperties.put(set.property, set.value)
 
       case unset: RemoveProperty =>
-        newProperties.remove(unset.property)
+        val prop = unset.property
+        if (!unset.ifExists && !properties.containsKey(prop) && prop != TableCatalog.PROP_COMMENT) {

Review comment:
       This is from:
   https://github.com/apache/spark/blob/e614f34c7a538b1f2c59616689eaea95af85fd54/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L307-L311




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571282171



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala
##########
@@ -116,7 +116,11 @@ private[sql] object CatalogV2Util {
         newProperties.put(set.property, set.value)
 
       case unset: RemoveProperty =>
-        newProperties.remove(unset.property)
+        val prop = unset.property
+        if (!unset.ifExists && !properties.containsKey(prop) && prop != TableCatalog.PROP_COMMENT) {

Review comment:
       This aligns with:
   https://github.com/apache/spark/blob/e614f34c7a538b1f2c59616689eaea95af85fd54/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L307-L311




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774340507


   **[Test build #134945 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134945/testReport)** for PR 31494 at commit [`dbbde92`](https://github.com/apache/spark/commit/dbbde92612ff4084905465174d027f8f13100679).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775651739


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39640/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775636072


   **[Test build #135058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135058/testReport)** for PR 31494 at commit [`8e4871a`](https://github.com/apache/spark/commit/8e4871a9054a6170c1ef37b9792e7832028d871e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775386321


   **[Test build #135044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135044/testReport)** for PR 31494 at commit [`5063e4e`](https://github.com/apache/spark/commit/5063e4e76431fb5b37bdc9ed33f6643b9e5cfa33).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571240319



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java
##########
@@ -64,7 +64,20 @@ static TableChange setProperty(String property, String value) {
    * @return a TableChange for the addition
    */
   static TableChange removeProperty(String property) {

Review comment:
       Can I directly update this API or no?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581291772



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Let me revert this commit first. We can start with the "change v1 to match v2" idea, as it's less breaking (some failed commands become noop).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774311918


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39524/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775528860


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135043/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775503100


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135044/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775452893


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39627/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777200079


   **[Test build #135110 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135110/testReport)** for PR 31494 at commit [`5438c55`](https://github.com/apache/spark/commit/5438c551ac48f676988ddf6873c2533f77eb3c5f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777277962


   **[Test build #135110 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135110/testReport)** for PR 31494 at commit [`5438c55`](https://github.com/apache/spark/commit/5438c551ac48f676988ddf6873c2533f77eb3c5f).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581531927



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Can we directly uses `r.table.properties.containsKey` ? In that way the `Table` implementation side returns a case (in)sensitive map via `properties` for case sensitivity. If they want them to treat table properties in a case insensitive way, I think it makes sense to return a case insensitive map from `Table.properties`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HyukjinKwon commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r573502020



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {

Review comment:
       Shall we add a JIRA prefix?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774375335


   cc @cloud-fan @MaxGekk 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774319136


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39524/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775757210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135058/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572333365



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {
+    val t = s"${catalogAndNamespace}table_name"
+    withTable(t) {
+      sql(s"CREATE TABLE $t (id int) USING $v2Format TBLPROPERTIES('test' = '34')")
+
+      val tableName = fullTableName(t)
+      val table = getTableMetadata(tableName)
+
+      assert(table.name === tableName)
+      assert(table.properties ===
+        withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava)
+
+      val exc = intercept[AnalysisException] {
+        sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('unknown')")
+      }
+      assert(exc.getMessage.contains("Attempted to unset non-existent property 'unknown'"))
+
+      // Reserved property "comment" should be allowed regardless.
+      sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('comment')")

Review comment:
       ah, I see!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775390813


   **[Test build #135043 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135043/testReport)** for PR 31494 at commit [`8b5508b`](https://github.com/apache/spark/commit/8b5508b2625f443362d17b377f6563fdfb897b77).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581318177



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Got it. Thank you, @cloud-fan .




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774290518


   **[Test build #134941 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134941/testReport)** for PR 31494 at commit [`0065170`](https://github.com/apache/spark/commit/0065170f24a2ff00cfee920e8fd31b0574f55ac7).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] rdblue commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581419810



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       I think that the new `ifExists` property in `RemoveProperty` would be easily overlooked or ignored. In general, I prefer fixing these problems in Spark and not adding requirements to the sources that will choose whether or not to follow them.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572353356



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {
+    val t = s"${catalogAndNamespace}table_name"
+    withTable(t) {
+      sql(s"CREATE TABLE $t (id int) USING $v2Format TBLPROPERTIES('test' = '34')")
+
+      val tableName = fullTableName(t)
+      val table = getTableMetadata(tableName)
+
+      assert(table.name === tableName)
+      assert(table.properties ===
+        withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava)
+
+      val exc = intercept[AnalysisException] {
+        sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('unknown')")
+      }
+      assert(exc.getMessage.contains("Attempted to unset non-existent property 'unknown'"))
+
+      // Reserved property "comment" should be allowed regardless.

Review comment:
       Yes, that's the current behavior of V1 command:
   https://github.com/apache/spark/blob/3b26bc25362a245a610c3e222b971b4ae612bc3e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L307-L314
   
   Since table comment is optional, it will just be set to `None` if the table had no comment:
   https://github.com/apache/spark/blob/3b26bc25362a245a610c3e222b971b4ae612bc3e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L316




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775468804






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572290672



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java
##########
@@ -64,7 +64,20 @@ static TableChange setProperty(String property, String value) {
    * @return a TableChange for the addition
    */
   static TableChange removeProperty(String property) {

Review comment:
       Yes, the analyzer way seems better. But the rule will handle the `ifExists = false` case and error out if the properties do not exist, instead of removing the table changes for the `ifExists = true` case; catalog implementation may want to log nonexistent properties to inform the user, for example.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774379652


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134941/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572292880



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving table properties such as handling non-existent properties.

Review comment:
       `... resolving AlterTableUnsetProperties such as ...`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] dongjoon-hyun closed pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun closed pull request #31494:
URL: https://github.com/apache/spark/pull/31494


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777278697


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135110/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775647271


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39640/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774353696






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774367636


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39528/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777211931


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39692/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774346760


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39527/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775386321


   **[Test build #135044 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135044/testReport)** for PR 31494 at commit [`5063e4e`](https://github.com/apache/spark/commit/5063e4e76431fb5b37bdc9ed33f6643b9e5cfa33).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775528860


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135043/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572291602



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving table properties such as handling non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {
+          throw new AnalysisException(
+            s"Attempted to unset non-existent property '$p' in table '${r.identifier.quoted}'")

Review comment:
       Note that we cannot remove the logic in https://github.com/apache/spark/blob/e614f34c7a538b1f2c59616689eaea95af85fd54/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L307-L311 because `ALTER VIEW` still needs it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775390813


   **[Test build #135043 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135043/testReport)** for PR 31494 at commit [`8b5508b`](https://github.com/apache/spark/commit/8b5508b2625f443362d17b377f6563fdfb897b77).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777278697


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135110/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774290518


   **[Test build #134941 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134941/testReport)** for PR 31494 at commit [`0065170`](https://github.com/apache/spark/commit/0065170f24a2ff00cfee920e8fd31b0574f55ac7).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775636072


   **[Test build #135058 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135058/testReport)** for PR 31494 at commit [`8e4871a`](https://github.com/apache/spark/commit/8e4871a9054a6170c1ef37b9792e7832028d871e).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581117990



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Just found a problem here: removing table property may be case insensitive (depends on the v2 table implementation), so the code here may break existing queries unexpectedly.
   
   I think the right approach is to change the v2 API and pass the `ifExists` flag to the v2 implementations. However, I feel like the current v2 behavior is OK (do not fail even if the property doesn't exist). So another idea is to change the v1 behavior instead, and ignore the `ifExists` flag and always do not fail.
   
   @imback82 @rdblue what do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572305204



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {
+    val t = s"${catalogAndNamespace}table_name"
+    withTable(t) {
+      sql(s"CREATE TABLE $t (id int) USING $v2Format TBLPROPERTIES('test' = '34')")
+
+      val tableName = fullTableName(t)
+      val table = getTableMetadata(tableName)
+
+      assert(table.name === tableName)
+      assert(table.properties ===
+        withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava)
+
+      val exc = intercept[AnalysisException] {
+        sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('unknown')")
+      }
+      assert(exc.getMessage.contains("Attempted to unset non-existent property 'unknown'"))
+
+      // Reserved property "comment" should be allowed regardless.
+      sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('comment')")

Review comment:
       The "comment" property seems to be treated differently that other reserved properties (not checked in the following):
   https://github.com/apache/spark/blob/3b26bc25362a245a610c3e222b971b4ae612bc3e/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala#L2836-L2850
   
   The following works:
   ```
   scala> sql("CREATE TABLE t (id int) TBLPROPERTIES('comment' = 'hello')")
   scala> sql("DESCRIBE EXTENDED t").show
   +--------------------+--------------------+-------+
   |            col_name|           data_type|comment|
   +--------------------+--------------------+-------+
   |                  id|                 int|   null|
   |                    |                    |       |
   |# Detailed Table ...|                    |       |
   |            Database|             default|       |
   |               Table|                   t|       |
   |               Owner|              terryk|       |
   |        Created Time|Mon Feb 08 11:08:...|       |
   |         Last Access|             UNKNOWN|       |
   |          Created By|         Spark 3.0.1|       |
   |                Type|             MANAGED|       |
   |            Provider|                hive|       |
   |             Comment|               hello|       |
   ```
   
   Note the following to see how v1 command can unset the comment using this command:
   https://github.com/apache/spark/blob/3b26bc25362a245a610c3e222b971b4ae612bc3e/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala#L315-L316




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775468804






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777242881


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39692/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572291464



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving table properties such as handling non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>

Review comment:
       Can we turn it into `NoopCommand` if possible?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774350161


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39528/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774394722


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134945/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581319932



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       Code-wise, yes, adding a new field to `RemoveProperty` is less breaking as it's binary compatible. However, we also changed the semantic of `RemoveProperty` and force all v2 implementations to update and respect the new `ifExists` flag, which is very breaking.
   
   I'd like to also point out that, the current v2 behavior is idempotent, you can run `UNSET TBLPROPERTIES` more than once and the result won't change. It's a good property and is consistent with `SET TBLPROPERTIES`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775503100


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135044/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572005786



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java
##########
@@ -64,7 +64,20 @@ static TableChange setProperty(String property, String value) {
    * @return a TableChange for the addition
    */
   static TableChange removeProperty(String property) {

Review comment:
       Yea we need to keep backward compatibility




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777200079


   **[Test build #135110 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135110/testReport)** for PR 31494 at commit [`5438c55`](https://github.com/apache/spark/commit/5438c551ac48f676988ddf6873c2533f77eb3c5f).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775746038


   **[Test build #135058 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135058/testReport)** for PR 31494 at commit [`8e4871a`](https://github.com/apache/spark/commit/8e4871a9054a6170c1ef37b9792e7832028d871e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-777251620


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39692/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774322983


   **[Test build #134944 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134944/testReport)** for PR 31494 at commit [`c757b6d`](https://github.com/apache/spark/commit/c757b6d7f2f8590fbe5f33e3a3c86d1968a4bb65).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774394722


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/134945/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774336115


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39524/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774340507


   **[Test build #134945 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134945/testReport)** for PR 31494 at commit [`dbbde92`](https://github.com/apache/spark/commit/dbbde92612ff4084905465174d027f8f13100679).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775651739


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39640/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775645976


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39640/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774371350


   **[Test build #134941 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134941/testReport)** for PR 31494 at commit [`0065170`](https://github.com/apache/spark/commit/0065170f24a2ff00cfee920e8fd31b0574f55ac7).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774367636


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39528/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774336115


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/39524/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775420690


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/39627/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572293623



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/AlterTableTests.scala
##########
@@ -1141,6 +1141,36 @@ trait AlterTableTests extends SharedSparkSession {
     }
   }
 
+  test("AlterTable: remove nonexistent table property") {
+    val t = s"${catalogAndNamespace}table_name"
+    withTable(t) {
+      sql(s"CREATE TABLE $t (id int) USING $v2Format TBLPROPERTIES('test' = '34')")
+
+      val tableName = fullTableName(t)
+      val table = getTableMetadata(tableName)
+
+      assert(table.name === tableName)
+      assert(table.properties ===
+        withDefaultOwnership(Map("provider" -> v2Format, "test" -> "34")).asJava)
+
+      val exc = intercept[AnalysisException] {
+        sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('unknown')")
+      }
+      assert(exc.getMessage.contains("Attempted to unset non-existent property 'unknown'"))
+
+      // Reserved property "comment" should be allowed regardless.
+      sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('comment')")

Review comment:
       hmm, IIRC we don't even allow users to set the reserved property in CREATE TABLE.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775757210


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/135058/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572292383



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving table properties such as handling non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>

Review comment:
       nvm, it's possible that someone else adds the table property before this command gets actually executed. It's safer to always execute this command.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-775502521


   **[Test build #135044 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/135044/testReport)** for PR 31494 at commit [`5063e4e`](https://github.com/apache/spark/commit/5063e4e76431fb5b37bdc9ed33f6643b9e5cfa33).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #31494:
URL: https://github.com/apache/spark/pull/31494#issuecomment-774353697






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581312258



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       If we want to go with a less breaking change, isn't updating V2 API a better option (you can check the change [here](https://github.com/apache/spark/pull/31494/commits/0065170f24a2ff00cfee920e8fd31b0574f55ac7#diff-ca6d2813b4735973d1fb343a7b19271074915ce44590445899a3e2ea88ae744eR78)) since we are just adding a new API?
   
   The only concern I have with "change v1 to match v2" idea is that users expect the behavior would be different if there exists an `IF EXISTS` option.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r581323742



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableProperties.scala
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.{AlterTableUnsetProperties, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.IdentifierHelper
+import org.apache.spark.sql.connector.catalog.TableCatalog
+
+/**
+ * A rule for resolving AlterTableUnsetProperties to handle non-existent properties.
+ */
+object ResolveTableProperties extends Rule[LogicalPlan] {
+  def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
+    case a @ AlterTableUnsetProperties(r: ResolvedTable, props, ifExists) if !ifExists =>
+      val tblProperties = r.table.properties.asScala
+      props.foreach { p =>
+        if (!tblProperties.contains(p) && p != TableCatalog.PROP_COMMENT) {

Review comment:
       OK, makes sense. Thanks.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] cloud-fan commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r572007541



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java
##########
@@ -64,7 +64,20 @@ static TableChange setProperty(String property, String value) {
    * @return a TableChange for the addition
    */
   static TableChange removeProperty(String property) {

Review comment:
       Another way is to do the work in Spark analyzer: if the key doesn't exist and `ifExists = true`, remove this table change.
   
   Then we don't need to change API.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571240319



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/TableChange.java
##########
@@ -64,7 +64,20 @@ static TableChange setProperty(String property, String value) {
    * @return a TableChange for the addition
    */
   static TableChange removeProperty(String property) {

Review comment:
       Can I directly update this API or do we need to keep the compatibility?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] imback82 commented on a change in pull request #31494: [SPARK-34380][SQL] Support ifExists for ALTER TABLE ... UNSET TBLPROPERTIES for v2 command

Posted by GitBox <gi...@apache.org>.
imback82 commented on a change in pull request #31494:
URL: https://github.com/apache/spark/pull/31494#discussion_r571342363



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -437,9 +437,8 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
       }.toSeq
       AlterTableExec(table.catalog, table.identifier, changes) :: Nil
 
-    // TODO: v2 `UNSET TBLPROPERTIES` should respect the ifExists flag.
-    case AlterTableUnsetProperties(table: ResolvedTable, keys, _) =>
-      val changes = keys.map(key => TableChange.removeProperty(key))
+    case AlterTableUnsetProperties(table: ResolvedTable, keys, ifExists) =>
+      val changes = keys.map(key => TableChange.removeProperty(key, ifExists))

Review comment:
       Another approach is do the existence check here instead of changing the public APIs.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org