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 2022/02/04 05:57:42 UTC

[GitHub] [spark] aokolnychyi opened a new pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

aokolnychyi opened a new pull request #35395:
URL: https://github.com/apache/spark/pull/35395


   <!--
   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'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### 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 contains changes to rewrite DELETE operations for V2 data sources that can replace groups of data (e.g. files, partitions).
   
   ### 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.
   -->
   
   These changes are needed to support row-level operations in Spark per SPIP SPARK-35801.
   
   ### 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, this PR adds new Data Source V2 APIs.
   
   ### 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.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   This PR comes with tests.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       I think the builder is the right way to go. The reason we opt for builders is so that extensions don't require deprecating parts of the API we need to change. If we ever need to add optional configuration, this should be a builder.
   
   There are some examples that we may want to introduce later. For example, if Spark wanted to introduce an isolation level setting, then we would most likely add it here. Same for things like group vs delta modes. Deciding when to use deltas vs rewrite files could easily be something that we extend here.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       This is the place where the writer can obtain unique groups IDs that were read (i.e. files in case of Delta).
   
   Those group IDs can be filtered in two ways:
   - static
   - runtime
   
   The proposal is that Spark will first should push down whatever possible conditions so that data sources can use static filters to prune partitions and files using metadata. You are right subqueries won't be pushed down but there may be other simple conditions. That gives us a set of files that **_may_** potentially match (still has false positives). In the example you provided, static filtering is not possible since we only have a subquery. Next, Spark should scan the potentially matching files using a separate filtering query to reduce the amount of data to rewrite. That should give us the minimum set of data files to rewrite. I agree with the filtering subquery you provided.
   
   I think we are talking about the same algorithm but I still believe the query you mentioned can be executed dynamically and the unique files can be collected and passed as a runtime IN filter.
   
   Let me show how plans can look like for a DELETE statement with an IN subquery.
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   Just like you said, the filtering subquery would be planned as a scalable left semi join.
   
   Does that make sense? I think we are talking about the same algorithm, just different representation.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       I don't think the result will be discarded. Let me explain how I see this:
   
   - The data source plans input splits for groups that may have matches using parts of the condition that can be converted into data source filters. 
   - The data source scan can cache group IDs or input splits that may have matches and expose filter attributes for runtime filtering. In case of Delta, the scan needs to remember a set of files that potentially have rows to update/delete.
   - If the data source supports runtime filtering, Spark will assign a filtering subquery that will be executed at runtime based on the SQL command. The same scan object will be used in the filtering subquery and in the row-level operation. In case of Delta, the runtime filtering will be done using the `_file_name` metadata column.
   - Spark executes the filtering subquery via the existing runtime filtering mechanism, collects unique values for the filtering attributes and passes them into the scan. In case of Delta, the scan will receive a set of files that have matches.
   - The data source uses the passed unique values to filter out cached group IDs or tasks and reports back input splits for groups that definitely have matches. In case of Delta, it can filter out previously cached files and include tasks only for files that have matches.
   - Writes have access to scans, so each scan can report a list of affected group IDs (which was cached). It can be in any format: strings, case classes, etc. Whatever a particular data source needs.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       I believe the current behavior in Delta pretty much matches the above algorithm.
   
   Before considering alternative designs, I wanted to make sure we agree the proposed APIs would work for Delta.
   Are we on the same page, @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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       We also may not be able to get rid of the optional rewrite plan in `DeleteFromTable`. Currently, I pass `DeleteFromTable` as it holds the actual delete condition that is pushed down during job planning in the optimizer. We cannot push down the condition in the rewrite plan as it has been negated for sources that replace groups. In other words, job planning for row-level commands is special and we cannot get rid of `DeleteFromTable` in the analyzer as it provides the required context to plan the job (hence a special rule for job planning).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I do have a working prototype for runtime filtering on top of the current implementation that adds the filter subquery to `ReplaceData` and requires an optimizer rule with 50 lines of code. We can use it as a starting point for runtime filtering but I am not sure it should be a blocker to this PR.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I do have a working prototype for runtime filtering on top of the current implementation that adds the filter subquery to `ReplaceData` and requires an optimizer rule with 50 lines of code. We can use it as a starting point for runtime filtering but I am not sure it should block this PR.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw QueryCompilationErrors.tableDoesNotSupportDeletesError(t)
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)

Review comment:
       Metadata columns may be used for multiple purposes:
   - grouping data on write
   Data sources may reference metadata columns in `RequiresDistributionAndOrdering` in the main write. In order to support this, such metadata columns must be included in the main scan. The comment above mentions potentially clustering data by shard/host before writing in JDBC data sources (just an example).
   
   - runtime filtering
   Data sources may reference metadata columns in `SupportsRuntimeFiltering` in the main scan. The idea we discussed earlier is that we build the main scan, it exposes `_file_name` as it is runtime filtering attributes, an optimizer rule assigns a filter subquery that uses another scan builder (i.e. for runtime filtering), Spark executes the subquery, collects unique values for `_file_name` and passes the results back to the main scan as an IN filter on `_file_name`.
   
   In both cases, the metadata columns are [projected away](https://github.com/apache/spark/pull/35395/files#diff-bc1d55a3b0b11d1ae3191b91a1d55d6e553d6974b0d8ffb4dc553d7feaa08b82R102) once the write is built and are never passed to the writer.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Do we have a test for this new rule?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       Thanks. @aokolnychyi I agree with you. Based on the discussion on the JIRA, I think @kazuyukitanimura will fix this soon.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Thanks for the explanation. Is it possible to change the rule `OptimizeMetadataOnlyDeleteFromTable` to make it also apply to `DeleteFromTable(relation, condition, None)`, and move the current check to there, and then use `DeleteFromTableWithFilters`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
##########
@@ -388,3 +391,62 @@ object ExtractSingleColumnNullAwareAntiJoin extends JoinSelectionHelper with Pre
     case _ => None
   }
 }
+
+/**
+ * An extractor for operations such as DELETE and MERGE that require rewriting data.
+ *
+ * This class extracts the following entities:
+ *  - the row-level command (such as [[DeleteFromTable]]);
+ *  - the read relation in the rewrite plan that can be either [[DataSourceV2Relation]] or
+ *  [[DataSourceV2ScanRelation]] depending on whether the planning has already happened;
+ *  - the current rewrite plan.

Review comment:
       It would be good to mention why this always finds the read relation rather than constructing the `RowLevelCommand` with a hard reference to it. My understanding is that it may be changed by the optimizer. It could be removed based on the condition and there may be more than one depending on the planning for UPDATE queries. Is that right?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)

Review comment:
       I am going to use that operation object to pick up a rewrite method later.
   
   ```
   val rewritePlan = operation match {
     case _: SupportsDelta =>
        buildWriteDeltaPlan(r, table, cond)
     case _ =>
       buildReplaceDataPlan(r, table, cond)
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }
+
+  protected def buildReadRelation(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      metadataAttrs: Seq[AttributeReference],
+      rowIdAttrs: Seq[AttributeReference] = Nil): DataSourceV2Relation = {

Review comment:
       I wouldn't say it's a new concept, it's the row ID fields from the SPIP.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       +1 for separate rules. The other one is complicated to allow extra pushdown that isn't needed here.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   I agree it may be beneficial to plan the filtering subquery differently if the job planning is fast. I am open to exposing a different scan builder for runtime filtering. Then data sources may choose to reuse the same scan if they want to but Spark will be flexible. We can discuss it one more time on the PR for runtime filtering. For now, a separate scan builder seems a better option.
   
   In either case, using `SupportsRuntimeFiltering` should work well and will be backward compatible with the current API. What do you think, @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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   @aokolnychyi, that plan sounds reasonable to me. Is there any expectation that information like scan predicates are shared between the two scans, or is that all handled by Spark?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       +1




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1074348488


   @rdblue @cloud-fan, I assumed the delete condition (not negated) would be explicitly passed to both scan builders by Spark. For instance, if the delete condition is `part_col = 'a' and id =1`, Spark would push it to the main scan builder and then provide an extra predicate on the filter attributes (e.g. `_file_name IN (...)`). Since the scan condition will be the same, data sources may cache and reuse some information between the scans.
   
   I can also see data sources delaying the actual split planning in the main scan up until they receive the runtime filter too. I guess there is a number of ways data sources can behave.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       Let me know if you agree with my analysis.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yeah, the API would definitely work with subqueries. We won't even have to do much to support them. The example above contained a subquery and was rewritten correctly (I executed my prototype).
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   > How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select _file_name column to get the "group id". Are we going to add an implicit assumption that, the Scan will add an extra column silently like my proposal? If yes, I think this solves the problem and is actually very similar to my proposal with the difference that we use the existing hidden column and runtime filter API to report "group id" from the source and use it to determine the final affected "groups".
   
   Delta's `Scan` for row-level operations can inherit `SupportsRuntimeFiltering` and report `_file_name` as a filtering attribute via `SupportsRuntimeFiltering$filterAttributes`. The `_file_name` should be a metadata column that can be projected via `RowLevelOperation$requiredMetadataAttributes`. That's what I propose as a way to report which column to use to filter out group IDs. Both metadata columns and runtime filtering is already supported for DS V2.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       I don't think the result will be discarded. Let me explain how I see this:
   
   - The data source plans input splits for groups that may have matches using parts of the condition that can be converted into data source filters. 
   - The data source can cache group IDs or input splits that may have matches and expose filter attributes for runtime filtering. In case of Delta, it needs to remember a set of files that potentially have rows to update/delete.
   - If the data source supports runtime filtering, Spark will assign a filtering subquery that will be executed at runtime based on the SQL command. The same scan object will be used in the filtering subquery and in the row-level operation. In case of Delta, the runtime filtering will be done using the `_file_name` metadata column.
   - Spark executes the filtering subquery via the existing runtime filtering mechanism, collects unique values for the filtering attributes and passes them into the scan. In case of Delta, the scan will receive a set of files that have matches.
   - The data source uses the passed unique values to filter out cached group IDs or tasks and reports back input splits for groups that definitely have matches. In case of Delta, it can filter out previously cached files and include tasks only for files that have matches.
   - Writes have access to scans, so each scan can report a list of affected group IDs (which was cached). It can be in any format: strings, case classes, etc. Whatever a particular data source needs.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())

Review comment:
       I am not sure. These are options passed into `newRowLevelOperationBuilder` and I thought they should come from the SQL operation. For example, if Spark adds a clause `OPTIONS` to its SQL for DELETE, UPDATE, MERGE, then these values will be propagated here.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>
+      relation.table match {
+        case table: SupportsDelete if !SubqueryExpression.hasSubquery(cond) =>
+          val predicates = splitConjunctivePredicates(cond)
+          val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, relation.output)
+          val filters = toDataSourceFilters(normalizedPredicates)
+          val allPredicatesTranslated = normalizedPredicates.size == filters.length
+          if (allPredicatesTranslated && table.canDeleteWhere(filters)) {
+            logDebug(s"Switching to delete with filters: ${filters.mkString("[", ", ", "]")}")
+            DeleteFromTableWithFilters(relation, filters)
+          } else {
+            d
+          }
+
+        case _: TruncatableTable if cond == TrueLiteral =>

Review comment:
       This will be triggered when a condition gets optimized into a true literal. The rewrite rule is invoked in the analyzer but this logic is part of the optimizer.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1074348488


   @rdblue @cloud-fan, I assumed the delete condition (not negated) would be explicitly passed to both scan builders by Spark. For instance, if the delete condition is `part_col = 'a' and id =1`, Spark would push it to the main scan builder and then provide an extra predicate on the filter attributes (e.g. `_file_name IN (...)`). Since the scan condition will be the same, data sources may cache and reuse some information between the scans. I can see data sources delaying the actual split planning in the main scan up until they receive the runtime filter too. I guess there is a number of ways data sources can behave.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1070968323


   > I think we can achieve that by reusing the same Scan object in the main and runtime filtering queries.
   
   This may be suboptimal because it's two different scans, with different column pruning and predicate pushdown. The scan to collect group ids only needs to return columns referred by the DELETE condition and the hidden column, while the scan to get the input data requires all the columns. Different predicate pushdown is also an important factor. Some data sources have multiple levels to handle predicate, such as file level and parquet row group in Delta Lake. We need to create a fresh `ScanBuilder` to apply dedicated column pruning and predicate pushdown required by the query.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   Alright, there are multiple ways to support a separate scan builder for runtime filtering.
   
   One way could be to expose a mix-in interface for `RowLevelOperation` with the following method (naming TBD):
   
   ```
   ScanBuilder newRuntimeFilterScanBuilder(CaseInsensitiveStringMap options);
   ```
   
   Then we can have an optimizer rule what will be applied after the main `Scan` for the row-level operation is built. That rule will catch row-level operations that implement the mix-in interface and where the main `Scan` extends `SupportsRuntimeFitlering`. The main `Scan` would tell us filter attributes in `filterAttributes`. During runtime filtering, we will collect unique values for these attributes and pass them back to the main `Scan` via `filter(Filter[] filters)`. The rule can construct a filter subquery that reference `DataSourceV2Relation`.  We will have to call `OptimizeSubquery` on it to rewrite predicate subqueries as joins anyway so we can leverage `V2ScanRelationPushDown` to do the planning that would use the runtime filter scan builder.
   
   This will enable schema pruning and filter pushdown within groups during runtime filtering.
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#135, dep#136] testhadoop.default.table
   +- Project [id#135, dep#136]
      +- Filter NOT (exists#157 <=> true)
         +- Join ExistenceJoin(exists#157), (id#135 = value#46)
            :- Project [id#135, dep#136]
            :  +- Filter dynamicpruning#154 [_file_name#139]
            :     :  +- Project [_file_name#152]
            :     :     +- Join LeftSemi, (id#150 = value#141)
            :     :        :- RelationV2[id#150, _file_name#152] testhadoop.default.table
            :     :        +- LocalRelation [value#141]
            :     +- RelationV2[id#135, dep#136, _file_name#139] testhadoop.default.table
            +- LocalRelation [value#46]
   ```
   
   In the example above, the scan relation in the runtime filter projects only `id` and `_file_name`.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   Sounds good, @cloud-fan @rdblue.
   
   Seems like we discussed all open points and reached consensus. Let me know if I missed anything. In the meantime, I'll split this PR into a few smaller ones and submit them separately.
   
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] jackylee-ch commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
jackylee-ch commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1078618084


   > I've rebased this PR and went through the code and comments one more time. This PR should be ready for another review round.
   
   There is one test failed on GA, maybe fix it first?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       `ReplaceData` is `NestedCommand` Here is an example how it is handled.
   
   ```
   sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
   ```
   
   ```
   == Parsed Logical Plan ==
   'DeleteFromTable ('id <= 1)
   +- 'UnresolvedRelation [cat, ns1, test_table], [], false
   
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table, org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- Project [id#88, dep#89]
      +- Sort [_partition#91 ASC NULLS FIRST], false
         +- RepartitionByExpression [_partition#91], 5
            +- Filter NOT ((id#88 <= 1) <=> true)
               +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Physical Plan ==
   ReplaceData org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- AdaptiveSparkPlan isFinalPlan=false
      +- Project [id#88, dep#89]
         +- Sort [_partition#91 ASC NULLS FIRST], false, 0
            +- Exchange hashpartitioning(_partition#91, 5), REPARTITION_BY_NUM, [id=#182]
               +- Project [id#88, dep#89, _partition#91]
                  +- Filter NOT ((id#88 <= 1) <=> true)
                     +- BatchScan[id#88, dep#89, _partition#91] class org.apache.spark.sql.connector.catalog.InMemoryTable$InMemoryBatchScan RuntimeFilters: []
   
   ```
   
   Originally, `ReplaceData` is nested in `DeleteFromTable`. We need to execute that plan only if the table does not support DELETEs with filters.  Currently, `ReplaceData` becomes a top-level node in the optimizer but I will try to move that to the physical planning (i.e. `DataSourceV2Strategy`).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRowLevelOperations.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.connector.catalog;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperationBuilder;
+import org.apache.spark.sql.connector.write.RowLevelOperation;
+import org.apache.spark.sql.connector.write.RowLevelOperationInfo;
+
+/**
+ * A mix-in interface for {@link Table} row-level operations support. Data sources can implement
+ * this interface to indicate they support rewriting data for DELETE, UPDATE, MERGE operations.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface SupportsRowLevelOperations extends Table {
+  /**
+   * Returns a {@link RowLevelOperationBuilder} to build a {@link RowLevelOperation}.
+   * Spark will call this method while planning DELETE, UPDATE and MERGE operations
+   * that require rewriting data.
+   *
+   * @param info the row-level operation info such command (e.g. DELETE) and options

Review comment:
       Yep, will fix.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kazuyukitanimura commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   > The test failure seem unrelated.
   > 
   > ```
   > annotations failed mypy checks:
   > [23](https://github.com/aokolnychyi/spark/runs/5102950901?check_suite_focus=true#step:15:23)
   > python/pyspark/ml/stat.py:478: error: Item "None" of "Optional[Any]" has no attribute "summary"  [union-attr]
   > [24](https://github.com/aokolnychyi/spark/runs/5102950901?check_suite_focus=true#step:15:24)
   > Found 1 error in 1 file (checked 324 source files)
   > ```
   
   I see that there is a revert commit in the upstream https://github.com/apache/spark/commit/e34d8eec019a0e60576fa7d6d2193d8a3c5bedab


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   The test failure seem unrelated.
   
   ```
   annotations failed mypy checks:
   [23](https://github.com/aokolnychyi/spark/runs/5102950901?check_suite_focus=true#step:15:23)
   python/pyspark/ml/stat.py:478: error: Item "None" of "Optional[Any]" has no attribute "summary"  [union-attr]
   [24](https://github.com/aokolnychyi/spark/runs/5102950901?check_suite_focus=true#step:15:24)
   Found 1 error in 1 file (checked 324 source files)
   ```


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
##########
@@ -37,7 +37,11 @@ class SparkOptimizer(
 
   override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
     // TODO: move SchemaPruning into catalyst
-    SchemaPruning :: V2ScanRelationPushDown :: V2Writes :: PruneFileSourcePartitions :: Nil
+    SchemaPruning :: RowLevelCommandScanRelationPushDown :: V2ScanRelationPushDown :: V2Writes ::

Review comment:
       Is it possible to merge `RowLevelCommandScanRelationPushDown` into `V2ScanRelationPushDown` so they're in one place?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }
+
+  protected def buildReadRelation(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      metadataAttrs: Seq[AttributeReference],
+      rowIdAttrs: Seq[AttributeReference] = Nil): DataSourceV2Relation = {

Review comment:
       what is `rowIdAttrs`? This looks like a new concept.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       I mention the proposed APIs for delta-based operations in the design [doc](https://docs.google.com/document/d/12Ywmc47j3l2WF4anG5vL4qlrhT2OKigb7_EbIKhxg60). We will need new APIs but they will extend the existing ones. They won't be entirely different.
   
   ```
   interface SupportsDelta extends RowLevelOperation {
     DeltaWriteBuilder newWriteBuilder(LogicalWriteInfo info);
     NamedReference[] rowId();
   }
   
   interface DeltaWriteBuilder extends WriteBuilder {
     DeltaWrite build();
   }
   
   interface DeltaWrite extends Write {
     DeltaBatchWrite toBatch();
   }
   
   interface DeltaBatchWrite extends BatchWrite {
     DeltaWriterFactory createBatchWriterFactory(...);
   }
   
   interface DeltaWriterFactory extends WriterFactory {
     DeltaWriter<InternalRow> createWriter(long partId, long taskId);
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }
+
+  protected def buildReadRelation(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      metadataAttrs: Seq[AttributeReference],
+      rowIdAttrs: Seq[AttributeReference] = Nil): DataSourceV2Relation = {

Review comment:
       I can remove for now it. It is for `SupportsDelta`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       Added a branch above.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.connector.write
+
+import java.util
+
+import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An internal v2 table implementation that wraps the original table and a logical row-level
+ * operation for DELETE, UPDATE, MERGE commands that require rewriting data.

Review comment:
       Added a note below.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I do have a working prototype for runtime filtering on top of the current implementation that adds the filter subquery to `ReplaceData` and requires an optimizer rule with 50 lines of code. We can use it as a starting point for runtime filtering but I would skip it in this PR.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       We actually have to prevent that (added the new rule to the list of rules that cannot be excluded).
   
   Here is how a DELETE command may look like.
   
   ```
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   ```
   
   The condition we should push down to the source is the DELETE condition `(id < 1)` (not the condition in the filter on top of the scan). Suppose we have a data source that can replace files. We have two files: File A contains IDs 1, 2, 3 and File B contains IDs 5, 6, 7. If we want to delete the record with ID = 1, we should push down the actual delete condition (ID = 1) for correct file pruning. Once the data source determines that only File A contains records to delete, we need to read the entire file and determine what records did not match the condition (that's what that filter on top of the scan is doing). Those records (IDs 2, 3 in our example) have to be written back to the data source as it can only replace files. That's why pushing the filter condition would actually be wrong and we have to prevent 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yeah, the API would definitely work with subqueries. We won't even have to do much to support them. The example above contained a subquery and was rewritten correctly (I executed my prototype).
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   > How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select _file_name column to get the "group id". Are we going to add an implicit assumption that, the Scan will add an extra column silently like my proposal? If yes, I think this solves the problem and is actually very similar to my proposal with the difference that we use the existing hidden column and runtime filter API to report "group id" from the source and use it to determine the final affected "groups".
   
   Delta's `Scan` for row-level operations can inherit `SupportsRuntimeFiltering` and report `_file_name` as a filtering attribute via `SupportsRuntimeFiltering$filterAttributes`. The `_file_name` should be a metadata column that can be projected via `RowLevelOperation$requiredMetadataAttributes`. That's what I propose as a way to report which column to use to filter out group IDs.
   
   Both metadata columns and runtime filtering is already supported for DS V2 so using runtime filtering with subqueries is a matter of adding an optimizer rule.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yeah, the API would definitely work with subqueries. We won't even have to do much to support them. The example above contained a subquery and was rewritten correctly (I executed my prototype).
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   > How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select _file_name column to get the "group id". Are we going to add an implicit assumption that, the Scan will add an extra column silently like my proposal? If yes, I think this solves the problem and is actually very similar to my proposal with the difference that we use the existing hidden column and runtime filter API to report "group id" from the source and use it to determine the final affected "groups".
   
   Delta's `Scan` for row-level operations can inherit `SupportsRuntimeFiltering` and report `_file_name` as a filtering attribute via `SupportsRuntimeFiltering$filterAttributes`. The `_file_name` should be a metadata column that can be projected via `RowLevelOperation$requiredMetadataAttributes`. That's what I propose as a way to report which column to use to filter out group IDs.
   
   Both metadata columns and runtime filtering is already supported for DS V2 so adding runtime filtering with subqueries is a matter of adding an optimizer rule.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       If a data source does not support replacing groups, it won't extend `SupportsRowLevelOperation` and we will fail in the analyzer.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] sunchao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())

Review comment:
       should we pass `options` from the V2 relation instead of just using `empty`?

##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data sources must produce all rows in a group
+   * if any are returned. Some sources will avoid pushing filters into files (file granularity),
+   * while others will avoid pruning files within a partition (partition granularity).
+   * <p>
+   * For example, if a source can only replace partitions, all rows from a partition must
+   * be returned by the scan, even if a filter can narrow the set of changes to a single file
+   * in the partition. Similarly, a source that can swap individual files must produce all rows
+   * of files where at least one record must be changed, not just the rows that must be changed.
+   */
+  ScanBuilder newScanBuilder(CaseInsensitiveStringMap options);

Review comment:
       curious: for non-delta based data sources, will this just share the same implementation with `SupportsRead.newScanBuilder`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>
+      relation.table match {
+        case table: SupportsDelete if !SubqueryExpression.hasSubquery(cond) =>
+          val predicates = splitConjunctivePredicates(cond)
+          val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, relation.output)
+          val filters = toDataSourceFilters(normalizedPredicates)
+          val allPredicatesTranslated = normalizedPredicates.size == filters.length
+          if (allPredicatesTranslated && table.canDeleteWhere(filters)) {
+            logDebug(s"Switching to delete with filters: ${filters.mkString("[", ", ", "]")}")
+            DeleteFromTableWithFilters(relation, filters)
+          } else {
+            d
+          }
+
+        case _: TruncatableTable if cond == TrueLiteral =>

Review comment:
       when this will be triggered? I think we don't assign rewrite plan in`RewriteDeleteFromTable`.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildRelationWithAttrs(relation, operationTable, metadataAttrs)
+
+    // construct a plan that contains unmatched rows in matched groups that must be carried over
+    // such rows do not match the condition but have to be copied over as the source can replace
+    // only groups of rows (e.g. if a source supports replacing files, unmatched rows in matched
+    // files must be carried over)
+    // it is safe to negate the condition here as RowLevelCommandScanRelationPushDown
+    // handles predicate pushdown for row-level operations in a special way
+    val remainingRowsFilter = Not(EqualNullSafe(cond, TrueLiteral))

Review comment:
       curious why we use `EqualNullSafe` here. What does it mean when `cond` evaluates to null?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       `Table ${t.name()} does not support DELETE statements`?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
##########
@@ -284,6 +284,22 @@ case class OverwritePartitionsDynamicExec(
     copy(query = newChild)
 }
 
+/**
+ * Physical plan node to replace data in existing tables.
+ */
+case class ReplaceDataExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    write: Write) extends V2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet

Review comment:
       nit: wonder if we need these for other case classes too.

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+/**
+ * A rule that builds a scan in row-level operations that require rewriting data.
+ *
+ * Note this rule must be run before [[V2ScanRelationPushDown]] as row-level operations must be
+ * planned in a special way.
+ */
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>
+      val table = relation.table.asRowLevelOperationTable
+      val condition = command.condition
+      val scanBuilder = table.newScanBuilder(relation.options)
+
+      val (pushedFilters, remainingFilters) = pushFilters(condition, relation.output, scanBuilder)
+      val pushedFiltersStr = if (pushedFilters.isLeft) {
+        pushedFilters.left.get.mkString(", ")
+      } else {
+        pushedFilters.right.get.mkString(", ")
+      }
+
+      val (scan, output) = PushDownUtils.pruneColumns(scanBuilder, relation, relation.output, Nil)
+
+      logInfo(
+        s"""
+           |Pushing operators to ${relation.name}
+           |Pushed filters: ${pushedFiltersStr}
+           |Filters that were not pushed: ${remainingFilters.mkString(",")}

Review comment:
       nit: maybe add a space after ,




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       What if the table implements `SupportsRowLevelOperations`? I think having optional condition in the delete node does not make much sense. It would be easier to default it to `TrueLiteral` during parsing. I'll try that out for now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Yes. I think I would prefer just setting the row-level plan to `None` in the optimizer. Is that the plan? (If so, then you could do that in this commit rather than following up with the `DeleteFromTableWithFilters` optimization)




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1075135376


   > could you elaborate a little bit more on why you think Spark should not push down the delete condition to the main scan builder?
   
   After a second thought, I think we can push down to the main `Scan`. Although the `_file_name IN (...)` predicate can exactly determine the groups, but pushing down the static predicate earlier to the main `Scan` doesn't hurt.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildRelationWithAttrs(relation, operationTable, metadataAttrs)
+
+    // construct a plan that contains unmatched rows in matched groups that must be carried over
+    // such rows do not match the condition but have to be copied over as the source can replace
+    // only groups of rows (e.g. if a source supports replacing files, unmatched rows in matched
+    // files must be carried over)
+    // it is safe to negate the condition here as RowLevelCommandScanRelationPushDown
+    // handles predicate pushdown for row-level operations in a special way
+    val remainingRowsFilter = Not(EqualNullSafe(cond, TrueLiteral))

Review comment:
       It is necessary to avoid cases when `null = true` evaluates to `null` and then gets further propagated.
   In this case, `null = true` should evaluate to false.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   > I think the proposed API should work well for Delta, Iceberg, and even Hive (where groups are partitions).
   
   I agree with this. The SPIP was well thought through and has already been voted on. Alternative proposals are great to hear and discuss as an improvement, but let's try to keep focused on implementing what the community agreed to.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       We can persist the actual DELETE/UPDATE/MERGE condition in `ReplaceData` and use that for job planning but the other concern about converting Catalyst expressions into data source filters without optimizing them remains.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kazuyukitanimura commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       PR submitted https://github.com/apache/spark/pull/35400




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       Well, I remember why I did this as an optimizer rule. It is because I wanted to avoid job planning when DELETE can be handled using filters.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Do you mean Spark just builds the `Filter` query, runs it and discards the result? The data source `Scan` should find a way to collect the affected "groups" by itself? I think it works, but it's a bit awkward/too implicit...




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I'm thinking about it a bit more. I think it's clearer that we compile group-based DELETE in two steps:
   1. create a plan to get affected groups
   2. create a plan to get the undeleted rows
   These two steps can share one `ScanBuilder` instance.
   
   First, we should have an analyzer rule to rewrite `DeleteFromTable`, similar to what this PR does. But we can make it simpler. We should just replace `DeleteFromTable` with a new plan
   ```
   case class GroupBasedReplaceData(
       table: NamedRelation,
       condition: Expression,
       affectedGroups: LogicalPlan,
       query: LogicalPlan,
       write: Option[Write] = None) extends V2WriteCommand with SupportsSubquery {
   }
   ```
   - The first 2 arguments are directly from `DeleteFromTable`, so that we can go back to `DeleteFromTable` later.
   - The `affectedGroups` is just a `Filter` with negated DELETE condition, on a DS v2 relation with `RowLevelOperationTable` that is introduced in this PR.
   - The `query` is almost the same as `affectedGroups` except that its `Filter` condition is the original DELETE condition.
   - The `affectedGroups` and `query` share the same `RowLevelOperationTable` instance, and `RowLevelOperationTable` should create `ScanBuilder` only once, e.g.
   ```
   private lazy val scanBuilder = table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty())
   override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = scanBuilder
   ```
   
   Second, we apply a new optimizer rule right before `V2ScanRelationPushDown`. This new rule does predicate pushdown for the `affectedGroups`, to configure `RowLevelOperationTable.scanBuilder`. In the future, when we add APIs to create dedicated v2 `Scan` to get affected groups, this rule can also evaluate the query and collect the affected groups. This rule can also go back to `DeleteFromTable` if we can.
   
   Next, the `V2ScanRelationPushDown` will just work and apply operator pushdown to `GroupBasedReplaceData.query`. Note that, predicate pushdown will be done again here, which is useful: the first predicate pushdown allows the data source to determine which groups to replace, the second predicate pushdown allows the data source to do pruning at runtime (like parquet row group pruning).
   
   Finally, we evaluate `GroupBasedReplaceDataExec`, similar to what this PR does.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       `ReplaceData` is `NestedCommand` Here is an example how it is handled.
   
   ```
       sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
   ```
   
   ```
   == Parsed Logical Plan ==
   'DeleteFromTable ('id <= 1)
   +- 'UnresolvedRelation [cat, ns1, test_table], [], false
   
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table, org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- Project [id#88, dep#89]
      +- Sort [_partition#91 ASC NULLS FIRST], false
         +- RepartitionByExpression [_partition#91], 5
            +- Filter NOT ((id#88 <= 1) <=> true)
               +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Physical Plan ==
   ReplaceData org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- AdaptiveSparkPlan isFinalPlan=false
      +- Project [id#88, dep#89]
         +- Sort [_partition#91 ASC NULLS FIRST], false, 0
            +- Exchange hashpartitioning(_partition#91, 5), REPARTITION_BY_NUM, [id=#182]
               +- Project [id#88, dep#89, _partition#91]
                  +- Filter NOT ((id#88 <= 1) <=> true)
                     +- BatchScan[id#88, dep#89, _partition#91] class org.apache.spark.sql.connector.catalog.InMemoryTable$InMemoryBatchScan RuntimeFilters: []
   
   ```
   
   Originally, `ReplaceData` is nested in `DeleteFromTable`. We need to execute that plan only if the table does not support DELETEs with filters.  Currently, `ReplaceData` becomes a top-level node in the optimizer but I will try to move that to the physical planning (i.e. `DataSourceV2Strategy`).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       Instead of having `ReplaceRewrittenRowLevelCommands` and `OptimizeMetadataOnlyDeleteFromTable` in the optimizer, we can add more logic to `DataSourceV2Strategy`. Let me try that.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kazuyukitanimura commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       The fix went in with https://github.com/apache/spark/pull/35428
   @aokolnychyi please rebase on master and try again.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I agree it would simplify the implementation but converting the delete condition into data source filters in the analyzer may be suboptimal. Right now, the optimizer rule I added runs after expression optimization, meaning the delete condition has been simplified and rewritten using static optimizer rules.
   
   I followed what we do for `OverwriteByExpression` that also constructs data source filters after expression optimization. That's is also what happens for deletes right now.
   
   To be clear, I am not a fan of adding nested commands or `DeleteFromTableWithFilters`. However, converting the delete condition in the analyzer has substantial limitations as well. Do we have other places that construct data source filters in the analyzer?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))

Review comment:
       Agreed. Will change.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       @cloud-fan, `DeleteFromTableWithFilters` is an optimization for `SupportsRowLevelOperations`. Existing deletes with filters would be unaffected. That being said, I am going to combine the existing logic in `DataSourceV2Strategy` with the optimizer rule I added, like discussed [here](https://github.com/apache/spark/pull/35395#discussion_r805292918). That way, we have the filter conversion logic just in one place. Let me know if you agree with that.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   Well, give me a day to think this through. I'll get back tomorrow.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       > Next, the V2ScanRelationPushDown will just work and apply operator pushdown to GroupBasedReplaceData.query. 
   
   I am afraid it is vice versa. The regular planning in `V2ScanRelationPushDown` can be applied to the `filter` subquery, not the `query` plan that contains remaining rows. The `query` plan contains a negated condition that cannot be pushed. That means the main scan must be fully built before `V2ScanRelationPushDown` kicks in. We cannot partially preconfigure the scan there. Parquet pushdown can be enabled in the `filter` subquery, not the main scan.
   
   > this rule can also evaluate the query and collect the affected groups
   
   If we execute the filter subquery in an optimizer rule to form an IN filter while building the main scan, we may not be able to benefit from reusing subqueries and other optimization as it will no longer be one plan. That’s why leveraging the existing framework for runtime filtering seemed so promising.
   
   Based on what we discussed, I removed nested commands by adding more information to `ReplaceData`. I think it simplifies the implementation a little bit. In the future, `ReplaceData` may also contain a filter subquery that will be assigned in the rewrite plan (just like you mentioned). However, let’s put the runtime filtering out of scope for now. We know we have multiple ways to do it: either evaluate a subquery in an optimizer rule or leverage the runtime filtering framework for DS V2. We want to support a custom scan builder anyway. I can work on that immediately after this PR.
   
   What do you think, @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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.connector.write
+
+import java.util
+
+import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An internal v2 table implementation that wraps the original table and a logical row-level
+ * operation for DELETE, UPDATE, MERGE commands that require rewriting data.
+ *
+ * The purpose of this table is to make the existing scan and write planning rules work
+ * with commands that require coordination between the scan and the write (so that the write
+ * knows what to replace).
+ */
+case class RowLevelOperationTable(

Review comment:
       I added `private[sql]` to `RowLevelOperationTable` to match `LogicalWriteInfoImpl` and other classes in this package.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }
+
+  protected def buildReadRelation(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      metadataAttrs: Seq[AttributeReference],
+      rowIdAttrs: Seq[AttributeReference] = Nil): DataSourceV2Relation = {

Review comment:
       Removed for now. Will add back once we support deltas.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       When I tried this, it actually uncovered all the differences. This rule has very specific purpose and covers only a subset of the logic in `DataSourceV2Strategy`. After a closer look, I am not sure throwing exceptions in the optimizer rule is a great idea. Since most steps are done through common utils (split predicates, normalization, filter conversion), I'd consider keeping this rule simple. It is main purpose is to remove the rewrite plan when a delete can be handled using filters.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,

Review comment:
       @viirya @rdblue @cloud-fan @huaxingao, a question. In all rules I added, having checks for None and Some(true), which are essentially the same thing, complicated the implementation. I made the condition required and default it to true in the parser. 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       Following previous question, how do we know if the data source can replace files? If it cannot, do we still should/need to push down the command filter?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Do you mean Spark just builds the `Filter` query, runs it and discards the result? The data source `Scan` should find a way to collect the affected "groups" by itself? I'm not sure how it works, the semi join is evaluated by Spark and how can the data source scan know which "group ids" get picked at the end.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yea we agree on the algorithm, the question is how the affected "groups" are determined and passed around. BTW, Delta does want to support conditions with subqueries, and we want to make sure the newly proposed API here can work with subqueries: https://github.com/delta-io/delta/issues/826
   
   > Spark executes the filtering subquery via the existing runtime filtering mechanism, collects unique values for the filtering attributes ...
   
   This is the key problem. How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select `_file_name` column to get the "group id". Are we going to add an implicit assumption that, the `Scan` will add an extra column silently like my proposal?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   > I'm afraid a unified physical plan execution may fit one very well but is awkward for the other.
   
   @cloud-fan, I am not proposing a unified execution plan. In fact, we should rewrite delta-based and group-based rewrites in a completely different way, just like you said. Whatever I have here is for group-based sources. I had an early prototype for delta-based sources in the old PR #33008. It has a separate execution path with `WriteDelta` instead of `ReplaceData`.
   
   > Figure out which "groups" are affected w.r.t. the DELETE/UPDATE/MERGE condition. Since the condition may contain subqueries, it's better to let Spark submit the job to evaluate the condition and let data source report the "groups", so that Spark can figure out which "groups" are affected. The execution path can be: Spark asks the data source to give an initial list of "groups" w.r.t. the condition, and then create a data source scan with the initial "groups". The data source scan has a hidden column to report the "group id", and spark runs the scan with the post-scan filters to collect the group ids.
   Scan these "groups" to get the new data after DELETE/UPDATE/MERGE, and write new data to new "groups".
   Commit the transaction to remove old "groups" and add new "groups".
   
   My proposal should support the use case you describe. Here is a quote from the design doc:
   
   > For example, sources that support replacing individual files may use static filters to prune partitions and files using metadata. This will allow them to find files that may have rows to delete/update. Since the metadata filtering is not precise and rewriting data is expensive, it may make sense to scan the potentially matching files for matches using a separate filtering query to reduce the amount of data to rewrite.
   
   I am proposing to leverage runtime filtering for this. It would just require a new optimizer rule and would work well with the proposed APIs in this PR. We can even include runtime filtering in 3.3 as DS V2 supports runtime filtering. I mentioned this in the SPIP doc. Here is how that optimizer rule can look like (it will assign a subquery to find matching groups).
   
   ```
   object RowLevelCommandDynamicPruning extends Rule[LogicalPlan] with PredicateHelper {
   
     override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
       // apply special dynamic filtering only for plans that don't support deltas
       case RewrittenRowLevelCommand(
           command: RowLevelCommand,
           DataSourceV2ScanRelation(_, scan: SupportsRuntimeFiltering, _),
           rewritePlan: ReplaceData) if conf.dynamicPartitionPruningEnabled && isCandidate(command) =>
   
         // use reference equality to find exactly the required scan relations
         val newRewritePlan = rewritePlan transformUp {
           case r: DataSourceV2ScanRelation if r.scan eq scan =>
             val pruningKeys = V2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
             val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
             Filter(dynamicPruningCond, r)
         }
         command.withNewRewritePlan(newRewritePlan)
     }
   }
   ```
   
   Sources like Delta can expose a metadata column `_file` and support dynamic filtering on it during row-level operations. I think that will be cleaner than having a notion of a group in the APIs.
   
   Tests I added also show how writes can access what "groups" were scanned and should be replaced. We don't have to pass replaced groups explicitly. For instance, `PartitionBasedOperation` added for testing remembers the scan that was used and the write has access to it. That's why scans can report to writes what groups must be replaced.
   
   The purpose of `RowLevelOperation` is to coordinate scans and writes.
   
   I think the proposed API should work well for Delta, Iceberg, and even Hive (where groups are partitions).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1067180811


   @aokolnychyi thanks for your patience to resolve my concerns! IIUC, the overall process of group-based DELETE is:
   1. Build a SQL query to calculate the groups that need to be replaced. The query scans the table, then filter by the DELETE condition, and finally select a hidden column that represents group id. This query can leverage the exiting optimizer framework to do filter pushdown, or even runtime filter pushdown if the DELETE condition has subqueries and this query is planned as a join.
   2. The data source writer gets the collected group ids, scan these groups, then filter by the opposite of the DELETE condition to get the data after deletion, and finally replace these groups.
   
   Note that, the above process scans some groups twice to get the minimum affected groups. We can also have a simplified process to only scan once but may overestimate the groups to be replaced:
   1. push down the DELETE condition to the source, so that it can estimate the groups that need to be replaced
   2. The data source writer gets the collected group ids, scan these groups, then filter by the opposite of the DELETE condition to get the data after deletion, and finally replace these groups.
   
   The major difference is, instead of running a SQL query to collect affected group ids, we can use the static filter from the DELETE condition to estimate the affected groups, which does not need to scan these groups.
   
   Now let's look at the proposed API. I think it works pretty well for the simplified process, and I believe you have a plan to support the non-simple process by adding extension APIs. Just to make sure it will be backward compatible and we are on the same page:
   1. We will add a subclass of `SupportsRowLevelOperaton`
   2. The subclass has a new method to return a `ScanBuilder` which is used to collect the affected groups accurately and has a hidden column to represent group id.
   3. `SupportsRowLevelOperaton.newScanBuilder` should return a `ScanBuilder` implementing `SupportsRuntimeFilter`, so that it can get the affected groups as an IN predicate.
   
   Other questions about this PR:
   1. row-level operation does not even call out the data writing behavior. Shall we consider other names like `SupportsDataUpdating`, `SupportsUPSERT`, et.?
   2. Why do we put the replace plan as a parameter of `DeleteFromTable`? Can we simply replace `DeleteFromTable` with the new replace plan in the rule?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)

Review comment:
       You might give a clearer example:
   
   ```scala
   // resolve all required metadata attrs that may be used for grouping data on write (e.g., by JDBC shard/host)
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      val partitionColumnRef = FieldReference(PartitionKeyColumn.name)
+      Array(partitionColumnRef)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(FieldReference(PartitionKeyColumn.name)))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              FieldReference(PartitionKeyColumn.name),
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryGroupBasedOperation"

Review comment:
       InMemoryPartitionReplaceOperation?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      val partitionColumnRef = FieldReference(PartitionKeyColumn.name)
+      Array(partitionColumnRef)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(FieldReference(PartitionKeyColumn.name)))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              FieldReference(PartitionKeyColumn.name),
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryGroupBasedOperation"

Review comment:
       InMemoryPartitionReplace?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       I think it would make sense to add the options to the method signature. Maybe default them to `empty()`, but still have a way to pass them in for later.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1074348488






-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   Sorry for not replying for a few days. Let me catch up on discussions that happened.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
##########
@@ -284,6 +284,22 @@ case class OverwritePartitionsDynamicExec(
     copy(query = newChild)
 }
 
+/**
+ * Physical plan node to replace data in existing tables.
+ */
+case class ReplaceDataExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    write: Write) extends V2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet

Review comment:
       Let me double check this.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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






-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly.

Review comment:
       Yes, we need to execute `ReplaceData` only if the delete with filters is not possible (which we can only check in the optimizer).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       I'll handle that with a new separate branch. No need to assign a rewrite plan.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      val partitionColumnRef = FieldReference(PartitionKeyColumn.name)

Review comment:
       Fixed.

##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      val partitionColumnRef = FieldReference(PartitionKeyColumn.name)
+      Array(partitionColumnRef)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(FieldReference(PartitionKeyColumn.name)))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              FieldReference(PartitionKeyColumn.name),
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryGroupBasedOperation"

Review comment:
       Fixed.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I know in general it's better to put optimizations in optimizer rules, but we can simplify the code quite a lot if we can do this in the rewrite rule `RewriteDeleteFromTable`.
   
   We can statically know if a v2 `Table` extends `SupportsDelete`, if the delete condition contain subquery or not, if the delete condition can be translated to data source filters. which means we don't need to rewrite `DeleteFromTable` when not necessary at the analysis phase.
   
   Then the rewrite rule can be quite simple:
   1. If the delete is metadata-only, keep the `DeleteFromTable` unchanged.
   2. Otherwise, if the table supports row-level operation, rewrite to `ReplaceData`
   
   This means we don't need to introduce the nested command concept, and we don't need to add a new field to `DeleteFromTable`, and we don't need the new logical plan `DeleteFromTableWithFilters`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.connector.write
+
+import java.util
+
+import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An internal v2 table implementation that wraps the original table and a logical row-level
+ * operation for DELETE, UPDATE, MERGE commands that require rewriting data.
+ *
+ * The purpose of this table is to make the existing scan and write planning rules work
+ * with commands that require coordination between the scan and the write (so that the write
+ * knows what to replace).
+ */
+case class RowLevelOperationTable(

Review comment:
       Good idea, I overlooked. 




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan, ReplaceData, RowLevelWrite}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * A rule that replaces a rewritten DELETE operation with a delete using filters if the data source
+ * can handle this DELETE command without executing the plan that operates on individual or groups
+ * of rows.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case RewrittenRowLevelCommand(rowLevelPlan, DELETE, cond, relation: DataSourceV2Relation) =>
+      relation.table match {
+        case table: SupportsDelete if !SubqueryExpression.hasSubquery(cond) =>
+          val predicates = splitConjunctivePredicates(cond)
+          val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, relation.output)
+          val filters = toDataSourceFilters(normalizedPredicates)
+          val allPredicatesTranslated = normalizedPredicates.size == filters.length
+          if (allPredicatesTranslated && table.canDeleteWhere(filters)) {
+            logDebug(s"Switching to delete with filters: ${filters.mkString("[", ", ", "]")}")
+            DeleteFromTableWithFilters(relation, filters)

Review comment:
       I construct `DeleteFromTableWithFilters` to avoid calling `canDeleteWhere` one more time later. For some data sources, checking if a delete using filters is possible is not cheap. The purpose of the new delete node is to avoid doing that work twice.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)

Review comment:
       That would work too. No preference from my side. I'll update.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       hmm, usually we add a builder API to configure the things it builds. But in this case, if we never expect to configure the  `RowLevelOperation` through this builder, do we really need this interface?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Something like this:
   
   ```
   // construct a filtering plan with the original scan relation
   val matchingRowsPlan = command match {
     case d: DeleteFromTable =>
       Filter(d.condition.get, relation)
   
     case u: UpdateTable =>
       Filter(u.condition.get, relation)
   
     case m: MergeIntoTable =>
       Join(relation, m.sourceTable, LeftSemi, Some(m.mergeCondition), JoinHint.NONE)
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       This is the place where the writer can obtain unique groups IDs that were read (i.e. files in case of Delta).
   
   Those group IDs can be filtered in two ways:
   - static
   - runtime
   
   The proposal is that Spark will first should push down whatever possible conditions so that data sources can use static filters to prune partitions and files using metadata. You are right subqueries won't be pushed down but there may be other simple conditions. That gives us a set of files that **_may_** potentially match (still has false positives). In the example you provided, static filtering is not possible since we only have a subquery. Next, Spark should scan the potentially matching files using a separate filtering query to reduce the amount of data to rewrite. That should give us the minimum set of data files to rewrite. I agree with the filtering subquery you provided.
   
   I think we are talking about the same algorithm but I still believe the query you mentioned can be executed dynamically and the unique files can be collected and passed as a runtime IN filter.
   
   Let me show how plans can look like for a DELETE statement with an IN subquery.
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   Just like you said, the filtering subquery would be planned as a scalable left semi join.
   
   Does that make sense? I think we are talking about the same algorithm, just different representations.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       This is the place where the writer can obtain unique groups IDs that were read (i.e. files in case of Delta).
   
   Those group IDs can be filtered in two ways:
   - static
   - runtime
   
   The proposal is that Spark will first should push down whatever possible conditions so that data sources can use static filters to prune partitions and files using metadata. You are right subqueries won't be pushed down but there may be other simple conditions. That gives us a set of files that **_may_** potentially match (still has false positives). In the example you provided, static filtering is not possible since we only have a subquery. Next, Spark should scan the potentially matching files using a separate filtering query to reduce the amount of data to rewrite. That should give us the minimum set of data files to rewrite. I agree with the filtering subquery you provided.
   
   I think we are talking about the same algorithm but I still believe the query you mentioned can be executed dynamically and the unique files can be collected and passed as runtime IN filter.
   
   Let me show how plans can look like for a DELETE statement with an IN subquery.
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   Just like you said, the filtering subquery would be planned as a scalable left semi join.
   
   Does that make sense? I think we are talking about the same algorithm, just different representation.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       Thank you both!




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRowLevelOperations.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.connector.catalog;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperationBuilder;
+import org.apache.spark.sql.connector.write.RowLevelOperation;
+import org.apache.spark.sql.connector.write.RowLevelOperationInfo;
+
+/**
+ * A mix-in interface for {@link Table} row-level operations support. Data sources can implement
+ * this interface to indicate they support rewriting data for DELETE, UPDATE, MERGE operations.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface SupportsRowLevelOperations extends Table {
+  /**
+   * Returns a {@link RowLevelOperationBuilder} to build a {@link RowLevelOperation}.
+   * Spark will call this method while planning DELETE, UPDATE and MERGE operations
+   * that require rewriting data.
+   *
+   * @param info the row-level operation info such command (e.g. DELETE) and options

Review comment:
       typo? "for" such command?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       Hmm, don't we need to pass any option map into the `RowLevelOperationInfo`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>

Review comment:
       Don't need to check `SupportsDelete`?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -176,6 +176,52 @@ object OverwritePartitionsDynamic {
   }
 }
 
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommand with NestedCommand {

Review comment:
       Can you add some param docs for these parameters?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       So we should not see such `NestedCommand` after optimization, right?

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       Where the optimizer will do the check and discard the rewrite plan? In `ReplaceRewrittenRowLevelCommands`, seems it simply gets the rewrite plan of the `RowLevelCommand` (e.g. `DeleteFromTable`).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>

Review comment:
       It is checked in the branch below. We should assign a rewrite plan only for `SupportsRowLevelOperations`, though.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -176,6 +176,52 @@ object OverwritePartitionsDynamic {
   }
 }
 
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommand with NestedCommand {

Review comment:
       Sure, will do.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kazuyukitanimura commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       Apologies for the delay. Testing a few more corner cases. I should be able to submit the PR today




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data data sources must produce all rows in a group

Review comment:
       Fixed.

##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationInfo.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * An interface with logical information for a row-level operation such as DELETE or MERGE.

Review comment:
       Added.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Yeah, I think that will be cleaner. Is everybody okay with moving the check from the physical planning to the optimizer?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       `ReplaceData` is `NestedCommand`. Here is an example how it is handled.
   
   ```
   sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
   ```
   
   ```
   == Parsed Logical Plan ==
   'DeleteFromTable ('id <= 1)
   +- 'UnresolvedRelation [cat, ns1, test_table], [], false
   
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table, org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- Project [id#88, dep#89]
      +- Sort [_partition#91 ASC NULLS FIRST], false
         +- RepartitionByExpression [_partition#91], 5
            +- Filter NOT ((id#88 <= 1) <=> true)
               +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Physical Plan ==
   ReplaceData org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- AdaptiveSparkPlan isFinalPlan=false
      +- Project [id#88, dep#89]
         +- Sort [_partition#91 ASC NULLS FIRST], false, 0
            +- Exchange hashpartitioning(_partition#91, 5), REPARTITION_BY_NUM, [id=#182]
               +- Project [id#88, dep#89, _partition#91]
                  +- Filter NOT ((id#88 <= 1) <=> true)
                     +- BatchScan[id#88, dep#89, _partition#91] class org.apache.spark.sql.connector.catalog.InMemoryTable$InMemoryBatchScan RuntimeFilters: []
   
   ```
   
   Originally, `ReplaceData` is nested in `DeleteFromTable`. We need to execute that plan only if the table does not support DELETEs with filters.  Currently, `ReplaceData` becomes a top-level node in the optimizer but I will try to move that to the physical planning (i.e. `DataSourceV2Strategy`).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)

Review comment:
       This operation is always accessed through the table. Would it make sense to change the helper method to `buildOperationTable` instead?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)

Review comment:
       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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)

Review comment:
       But could you just call `table.operation match { ... }` instead?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       Added.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Please, take another look and let me know what you think, @huaxingao @rdblue @cloud-fan @viirya.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly.

Review comment:
       oh, then `ReplaceData` will be replaced by `CommandResult` and break `DeleteFromTable`




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly.

Review comment:
       what happens if we eagerly execute `ReplaceData`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }
+
+  protected def buildReadRelation(
+      relation: DataSourceV2Relation,
+      table: RowLevelOperationTable,
+      metadataAttrs: Seq[AttributeReference],
+      rowIdAttrs: Seq[AttributeReference] = Nil): DataSourceV2Relation = {

Review comment:
       I can remove it for now. It is for `SupportsDelta`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       Yeah, we will need one mix-in trait to indicate that a row-level operation supports deltas. I give an example [here](https://github.com/apache/spark/pull/35395#discussion_r813124275).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   cc @huaxingao @dongjoon-hyun @sunchao @cloud-fan @viirya @rdblue @HyukjinKwon @dbtsai 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1070968323


   > I think we can achieve that by reusing the same Scan object in the main and runtime filtering queries.
   
   This may be suboptimal because it's two different scans, with different column pruning and predicate pushdown. The scan to collect group ids only needs to return columns referred by the DELETE condition and the hidden column, while the scan to get the input data requires all the columns. Different predicate pushdown is also an important factor. Some data sources have multiple levels to handle predicate, such as file level and parquet row group level in Delta Lake. We need to create a fresh `ScanBuilder` to apply dedicated column pruning and predicate pushdown required by the query.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,

Review comment:
       While I think it is better overall, I might not combine the DELETE FROM work with this change. It adds a lot of files to this diff.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1061516629


   Also cc @tdas @zsxwing 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Is this the place where the writer gets the affected "groups"?
   
   It seems to me that the assumption is, we can always push down the delete condition to the data source scan to get the affected groups, which requires two things:
   1. we can translate all the catalyst predicates to data source Filters.
   2. subqueries can also work as runtime filters.
   
   I think the first thing will be true eventually, but the second thing may not always be possible. For example, `DELETE FROM t1 WHERE t1.c IN (SELECT c FROM t2)`. We won't get a runtime filter if `t2` is very big.
   
   In my proposal, Spark is responsible to run a query and collect affected "groups". In Delta, the query for the above example is like: `SELECT DISTINCT _file_name FROM t1 WHERE t1.c IN (SELECT c FROM t2)`. This query will be planned as a left semi join, and works with big `t2` as well (use shuffle join instead of broadcast join).
   
   The general idea is, Spark should drive the calculating of affected "groups", and data source should just report the "group id". I think this is the only way to support all kinds of UPDATE/DELETE/MERGE conditions: let Spark evaluate the condition.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   @cloud-fan, could you elaborate a little bit more on why you think Spark should not push down the delete condition to the main scan builder?


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
viirya edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1075725605


   As people are done reviewing with this change and reached consensus, if the smaller ones are the same without change, do we need to split this to smaller ones? Just a question. Seems okay to merge this directly?
   
   @cloud-fan @rdblue 
   
   Update: seems the smaller ones got approvals quickly. then it's fine.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] jackylee-ch commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
jackylee-ch commented on a change in pull request #35395:
URL: https://github.com/apache/spark/pull/35395#discussion_r834919726



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Expression) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))

Review comment:
       nit: 
   ```
   newChildren match {
       case Seq(newChild) =>
           xxx
       case Seq(_, newChild) =>
           xxx
       case _ =>
           throw exception/error
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yeah, the API would definitely work with subqueries. We won't even have to do much to support them. The example above contained a subquery and was rewritten correctly.
   
   ```
   DELETE FROM t WHERE id IN (SELECT * FROM deleted_id)
   ```
   
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#66, dep#67] t
   +- Project [id#66, dep#67]
      +- Filter NOT (exists#86 <=> true)
         +- Join ExistenceJoin(exists#86), (id#66 = value#19)
            :- Project [id#66, dep#67]
            :  +- Filter dynamicpruning#85 [_file_name#70]
            :     :  +- Project [_file_name#83]
            :     :     +- Join LeftSemi, (id#81 = value#72)
            :     :        :- Project [id#81, _file_name#83]
            :     :        :  +- RelationV2[id#81, dep#82, _file_name#83] t
            :     :        +- LocalRelation [value#72]
            :     +- RelationV2[id#66, dep#67, _file_name#70] t
            +- LocalRelation [value#19]
   ```
   
   > How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select _file_name column to get the "group id". Are we going to add an implicit assumption that, the Scan will add an extra column silently like my proposal? If yes, I think this solves the problem and is actually very similar to my proposal with the difference that we use the existing hidden column and runtime filter API to report "group id" from the source and use it to determine the final affected "groups".
   
   Delta's `Scan` for row-level operations can inherit `SupportsRuntimeFiltering` and report `_file_name` as a filtering attribute via `SupportsRuntimeFiltering$filterAttributes`. The `_file_name` should be a metadata column that can be projected via `RowLevelOperation$requiredMetadataAttributes`. That's what I propose as a way to report which column to use to filter out group IDs. Both metadata columns and runtime filtering is already supported for DS V2.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw QueryCompilationErrors.tableDoesNotSupportDeletesError(t)
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)

Review comment:
       Metadata columns may be used for multiple purposes:
   - grouping data on write
   Data sources may reference metadata columns in `RequiresDistributionAndOrdering`. In order to support this, such metadata columns must be included in the main scan. The comment above mentions potentially clustering data by shard/host before writing in JDBC data sources (just an example).
   
   - runtime filtering
   Data sources may reference metadata columns in `SupportsRuntimeFiltering` in the main scan. The idea we discussed earlier is that we build the main scan, it exposes `_file_name` as it is runtime filtering attributes, an optimizer rule assigns a filter subquery that uses another scan builder (i.e. for runtime filtering), Spark executes the subquery, collects unique values for `_file_name` and passes the results back to the main scan as an IN filter on `_file_name`.
   
   In both cases, the metadata columns are [projected away](https://github.com/apache/spark/pull/35395/files#diff-bc1d55a3b0b11d1ae3191b91a1d55d6e553d6974b0d8ffb4dc553d7feaa08b82R102) once the write is built and are never passed to the writer.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan, ReplaceData, RowLevelWrite}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * A rule that replaces a rewritten DELETE operation with a delete using filters if the data source
+ * can handle this DELETE command without executing the plan that operates on individual or groups
+ * of rows.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case RewrittenRowLevelCommand(rowLevelPlan, DELETE, cond, relation: DataSourceV2Relation) =>
+      relation.table match {
+        case table: SupportsDelete if !SubqueryExpression.hasSubquery(cond) =>
+          val predicates = splitConjunctivePredicates(cond)
+          val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, relation.output)
+          val filters = toDataSourceFilters(normalizedPredicates)
+          val allPredicatesTranslated = normalizedPredicates.size == filters.length
+          if (allPredicatesTranslated && table.canDeleteWhere(filters)) {
+            logDebug(s"Switching to delete with filters: ${filters.mkString("[", ", ", "]")}")
+            DeleteFromTableWithFilters(relation, filters)

Review comment:
       I construct `DeleteFromTableWithFilters` here to avoid calling `canDeleteWhere` one more time later. For some data sources, checking if a delete using filters is possible is not cheap. The purpose of the new delete node is to avoid doing that work twice.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data data sources must produce all rows in a group

Review comment:
       nit: `data data` typo?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationInfo.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * An interface with logical information for a row-level operation such as DELETE or MERGE.

Review comment:
       nit: shall we also put `UPDATE` here?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       is it possible to push down the negated filter in the rewrite plan?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,

Review comment:
       Yeah, I can make a separate PR for this if everyone is OK.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1061514337


   Sorry for the late review as I've been struggling with the API design for a long time. I understand that we want to have a unified DS v2 API for delta-based and group-based data sources, but these two have very different execution paths and I'm afraid a unified physical plan execution may fit one very well but is awkward for the other.
   
   The execution path of group-based data sources (please correct me if my understanding is wrong):
   1. Figure out which "groups" are affected w.r.t. the DELETE/UPDATE/MERGE condition. Since the condition may contain subqueries, it's better to let Spark submit the job to evaluate the condition and let data source report the "groups", so that Spark can figure out which "groups" are affected. The execution path can be: Spark asks the data source to give an initial list of "groups" w.r.t. the condition, and then create a data source scan with the initial "groups". The data source scan has a hidden column to report the "group id", and spark runs the scan with the post-scan filters to collect the group ids.
   2. Scan these "groups" to get the new data after DELETE/UPDATE/MERGE, and write new data to new "groups".
   3. Commit the transaction to remove old "groups" and add new "groups".
   
   To fit this execution path, the API proposal from me is (naming is not finalized)
   ```
   interface SupportsGroupBasedUpsert extends Table {
     String[] getAffectedGroups(Filter[] condition);
     // This scan should put an extra string column at the end of each row to indicate the group id.
     ScanBuilder newScanBuilder(String[] groups, CaseInsensitiveStringMap options);
    // The created `Write` must exends `SupportReplaceGroups`
     WriteBuilder newWriteBuilder(CaseInsensitiveStringMap options);
   }
   ...
   interface SupportReplaceGroups extends Write {
     BatchWrite toBatch(String[] groupsToReplace);
   }
   ```
   Note: the API above assumes the group can always be represented as a string. If there are objections, we can use binary and add an API to serialize/deserialize groups. e.g.
   ```
   Group[] getAffectedGroups(Filter[] condition);
   Group fromBinary(byte[] binary);
   // This scan should put an extra binary column at the end of each row to indicate the group id.
   ScanBuilder newScanBuilder(Group[] groups, CaseInsensitiveStringMap options);
   ...
   interface Group extends Serializable {}
   ```
   We can also follow `ContinuousPartitionReader` if we want more type-safety and don't want to use hidden columns. But it will make the API much more complicated.
   
   The entire workflow can be
   1. The initial logical plan is still `DeleteFromTable`
   2. An optimizer rule turns `DeleteFromTable` into `GroupBasedDeleteFromTable`, which extracts translatable v2 `Filter`s from the catalyst condition and calls `getAffectedGroups` to get the initial group list. Then create v2 `Scan` with the initial group list.
   3. `GroupBasedDeleteFromTable` is defined as
   ```
   GroupBasedDeleteFromTable(
     SubqueryExpression(Distinct(Project(the_group_id_col, Filter(cond, V2Scan)))),
     Filter(Not(cond), V2Scan),
     V2Write)
   ```
   4. the planner rule turns `GroupBasedDeleteFromTable` into its corresponding physical plan
   5. The physical plan get the final group list from the subquery, create `BatchWrite` and write data, and finally commit the transaction.
   
   We can also update the rule `V2Writes` to add shuffle/sort for `GroupBasedDeleteFromTable`


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,

Review comment:
       I think it's a good change.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw QueryCompilationErrors.tableDoesNotSupportDeletesError(t)
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)

Review comment:
       I'm a little confused here. Why do we need to include metadata attributes to build the "main scan" which is the input of the write operation? According to our previous discussion, metadata attributes should be used to build the scan for collecting the affected groups, e.g. the `_file_name` metadata column,




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+/**
+ * A rule that builds a scan in row-level operations that require rewriting data.
+ *
+ * Note this rule must be run before [[V2ScanRelationPushDown]] as row-level operations must be
+ * planned in a special way.
+ */
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>
+      val table = relation.table.asRowLevelOperationTable
+      val condition = command.condition
+      val scanBuilder = table.newScanBuilder(relation.options)
+
+      val (pushedFilters, remainingFilters) = pushFilters(condition, relation.output, scanBuilder)
+      val pushedFiltersStr = if (pushedFilters.isLeft) {
+        pushedFilters.left.get.mkString(", ")
+      } else {
+        pushedFilters.right.get.mkString(", ")
+      }
+
+      val (scan, output) = PushDownUtils.pruneColumns(scanBuilder, relation, relation.output, Nil)
+
+      logInfo(
+        s"""
+           |Pushing operators to ${relation.name}
+           |Pushed filters: ${pushedFiltersStr}
+           |Filters that were not pushed: ${remainingFilters.mkString(",")}

Review comment:
       Fixed.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   As people are done reviewing with this change and reached consensus, if the smaller ones are the same without change, do we need to split this to smaller ones? Just a question. Seems okay to merge this directly?
   
   @cloud-fan @rdblue 


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I'm thinking about it a bit more. I think it's clearer if we resolve group-based DELETE in two steps:
   1. create a plan to get affected groups
   2. create a plan to get the undeleted rows
   
   These two steps can share one `ScanBuilder` instance.
   
   First, we should have an analyzer rule to rewrite `DeleteFromTable`, similar to what this PR does. But we can make it simpler. We should just replace `DeleteFromTable` with a new plan
   ```
   case class GroupBasedReplaceData(
       table: NamedRelation,
       condition: Expression,
       affectedGroups: LogicalPlan,
       query: LogicalPlan,
       write: Option[Write] = None) extends V2WriteCommand with SupportsSubquery {
   }
   ```
   - The first 2 arguments are directly from `DeleteFromTable`, so that we can go back to `DeleteFromTable` later.
   - The `affectedGroups` is just a `Filter` with negated DELETE condition, on a DS v2 relation with `RowLevelOperationTable` that is introduced in this PR.
   - The `query` is almost the same as `affectedGroups` except that its `Filter` condition is the original DELETE condition.
   - The `affectedGroups` and `query` share the same `RowLevelOperationTable` instance, and `RowLevelOperationTable` should create `ScanBuilder` only once, e.g.
   ```
   private lazy val scanBuilder = table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty())
   override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = scanBuilder
   ```
   
   Second, we apply a new optimizer rule right before `V2ScanRelationPushDown`. This new rule does predicate pushdown for the `affectedGroups`, to configure `RowLevelOperationTable.scanBuilder`. In the future, when we add APIs to create dedicated v2 `Scan` to get affected groups, this rule can also evaluate the query and collect the affected groups, and add an IN predicate to `query`. This rule can also go back to `DeleteFromTable` if we can.
   
   Next, the `V2ScanRelationPushDown` will just work and apply operator pushdown to `GroupBasedReplaceData.query`. Note that, predicate pushdown will be done again here, which is useful: the first predicate pushdown allows the data source to determine which groups to replace, the second predicate pushdown allows the data source to do pruning at runtime (like parquet row group pruning).
   
   Finally, we evaluate `GroupBasedReplaceDataExec`, similar to what this PR does.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       It looks to me that the "group" is totally hidden in both the API design and execution framework. e.g. here we push down the delete condition and assume the `Table` will return a `Scan` with only the affected groups, and then at execution time, the `Write` will only rewrite these affected groups.
   
   This is probably a fine assumption as both the `Scan` and `Write` are special, returned from `RowLevelOperation`. How are we going to support tables that can do delta-based DELETE/UPDATE/MERGE? Are we going to add a bunch of new 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       I defined the mix-in interface on `RowLevelOperation`, though.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       Well, Spark can already plan filter-based DELETE today, so not supporting it would become a regression.

##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       Well, Spark can already plan filter-based DELETE today, so not supporting it would be a regression.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       Can we match `Some(cond)` to avoid needing to do this? You could also have a rule above that matches None with `SupportsDelete` or `SupportsTruncate` to avoid the case where this is `true`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1070968323


   > I think we can achieve that by reusing the same Scan object in the main and runtime filtering queries.
   
   This may be suboptimal because it's two different scans, with different column pruning and predicate pushdown. The scan to collect group ids only needs to return columns referred by the DELETE condition and the hidden column, while the scan to get the input data requires all the columns. Different predicate pushdown is also an important factor. Some data sources have multiple levels to handle predicate, such as file level and parquet row group in Delta Lake. We need to create a fresh `ScanBuilder` to apply dedicated column pruning and predicate pushdown required by the query.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1070968323


   > I think we can achieve that by reusing the same Scan object in the main and runtime filtering queries.
   
   This may be suboptimal because it's two different scans, with different column pruning and predicate pushdown. The scan to collect group ids only needs to return columns referred by the DELETE condition and the hidden column, while the scan to get the input data requires all the columns. Different predicate pushdown is also an important factor. Some data sources have multiple levels to handle predicate, such as file level and parquet row group level in Delta Lake. We need to create a fresh `ScanBuilder` to apply dedicated column pruning and predicate pushdown required by the query.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   > I assumed the delete condition (not negated) would be explicitly passed to both scan builders by Spark
   
   Good, I would prefer not to assume that both scans use information passed to one or the other.
   
   > pushing down the static predicate earlier to the main Scan doesn't hurt.
   
   I agree. In many cases this is actually required. For example, in a Hive table you wouldn't want to try to reconstruct a partition filter out of paths from the `IN` predicate. So you'd would also want to do partition pruning before file pruning.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1075391858


   Sounds good, @cloud-fan @rdblue.
   
   Seems like we discussed all open points and reached consensus. Let me know if I missed anything. In the meantime, I'll split this PR into a few smaller ones and submit them separately.
   
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       The issue has been fixed. Resolving this thread now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       Spark only supports SQL DELETE statements and there is no way for a user to pass options, unfortunately. However, the data source APIs added in this PR assume there will be options in the future. They just happen to be empty right now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       @kazuyukitanimura @viirya, this test fails because `NotPropagation` introduced in SPARK-36665 rewrites the condition in a way that breaks `RewritePredicateSubquery`. I added more context to the original JIRA issue. In short, it seems the rule behaves correctly for NOT IN predicates but NOT IN subqueries have to be handled differently.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   Alright, the tests are green and the PR is ready for a detailed review.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] kazuyukitanimura commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       With the PR this test is passing locally.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       I changed the behavior slightly so the last comment is not accurate anymore.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       It is done in `OptimizeMetadataOnlyDeleteFromTable`. That rule checks if we can handle this delete by just passing data source filters. If yes, the rewrite plan is nullified.
   
   That being said, we will call `SupportsDelete$canDeleteWhere` two times now. The first call will be in the optimizer and  the second one will be in `DataSourceV2Strategy`. 




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Is this the same as `DeleteFromTableWithFilters`? Is it possible to combine these two?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] huaxingao commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data data sources must produce all rows in a group

Review comment:
       nit: `data data` typo?

##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationInfo.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * An interface with logical information for a row-level operation such as DELETE or MERGE.

Review comment:
       nit: shall we also put `UPDATE` here?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       Is this the same as `DeleteFromTableWithFilters`? Is it possible to combine these two?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Do we have a test for this new rule?

##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       is it possible to push down the negated filter in the rewrite plan?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       This optimizer rule contains logic similar to what we have in `DataSourceV2Strategy` but it is done in the optimizer to avoid building `Scan` and `Write` if a DELETE operation can be handled using filters.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       This is okay. I think you could probably add a pushdown function in the existing pushdown class that uses the `RewrittenRowLevelCommand` matcher but returns the `ScanBuilderHolder` that is now used. But since pushdown for the row-level rewrite commands is so specific, I think it's probably more readable and maintainable over time to use a separate rule like this.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/connector/write/RowLevelOperationTable.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.connector.write
+
+import java.util
+
+import org.apache.spark.sql.connector.catalog.{SupportsRead, SupportsRowLevelOperations, SupportsWrite, Table, TableCapability}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * An internal v2 table implementation that wraps the original table and a logical row-level
+ * operation for DELETE, UPDATE, MERGE commands that require rewriting data.

Review comment:
       I think this should mention that the purpose of this table is to make the existing scan and write planning rules work with commands that require coordination between the scan and the write (so that the write knows what to replace).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       What if the table only implements `SupportsRowLevelOperations`? I think having optional condition in the delete node does not make much sense. It would be easier to default it to `TrueLiteral` during parsing. I'll try that out for now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       This thread is a bit confusing now. I'll resolve it. There are a few others where we discuss the same thing.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       +1




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildOperationTable(

Review comment:
       I renamed the method and added `options`. I did not default it to an empty map to stay on one line.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       The current proposal is to leverage existing `SupportsRuntimeFiltering`. Data sources that support row-level operations will need to implement `RowLevelOperation` added in this PR. That one is used to instantiate a scan builder, which means data sources can provide a special scan for row-level operations. Runtime filtering in DS V2 is very flexible and can support metadata columns.
   
   In case of Delta, it should be sufficient to have a scan that would report `_file_name` as a filtering attribute and then Spark will execute a subquery and pass unique file names back into the scan. Since the write will have access to the scan, it will also know the set of files that was scanned and needs to be replaced with new files.
   
   I was trying to keep the row-level API as simple as possible and rely on existing concepts. This approach builds on top of metadata columns and runtime filtering we already support.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       In case of regular Hive tables, runtime filtering can still use data columns.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1061514337


   Sorry for the late review as I've been struggling with the API design for a long time. I understand that we want to have a unified DS v2 API for delta-based and group-based data sources, but these two have very different execution paths and I'm afraid a unified physical plan execution may fit one very well but is awkward for the other.
   
   The execution path of group-based data sources (please correct me if my understanding is wrong):
   1. Figure out which "groups" are affected w.r.t. the DELETE/UPDATE/MERGE condition. Since the condition may contain subqueries, it's better to let Spark submit the job to evaluate the condition and let data source report the "groups", so that Spark can figure out which "groups" are affected. The execution path can be: Spark asks the data source to give an initial list of "groups" w.r.t. the condition, and then create a data source scan with the initial "groups". The data source scan has a hidden column to report the "group id", and spark runs the scan with the post-scan filters to collect the group ids.
   2. Scan these "groups" to get the new data after DELETE/UPDATE/MERGE, and write new data to new "groups".
   3. Commit the transaction to remove old "groups" and add new "groups".
   
   To fit this execution path, the API proposal from me is (naming is not finalized)
   ```
   interface SupportsGroupBasedUpsert extends Table {
     String[] getAffectedGroups(Filter[] condition);
     // This scan should put an extra string column at the end of each row to indicate the group id.
     ScanBuilder newScanBuilder(String[] groups, CaseInsensitiveStringMap options);
    // The created `Write` must exends `SupportReplaceGroups`
     WriteBuilder newWriteBuilder(CaseInsensitiveStringMap options);
   }
   ...
   interface SupportReplaceGroups extends Write {
     BatchWrite toBatch(String[] groups);
   }
   ```
   Note: the API above assumes the group can always be represented as a string. If there are objections, we can use binary and add an API to serialize/deserialize groups. e.g.
   ```
   Group[] getAffectedGroups(Filter[] condition);
   Group fromBinary(byte[] binary);
   // This scan should put an extra binary column at the end of each row to indicate the group id.
   ScanBuilder newScanBuilder(Group[] groups, CaseInsensitiveStringMap options);
   ...
   interface Group extends Serializable {}
   ```
   We can also follow `ContinuousPartitionReader` if we want more type-safety and don't want to use hidden columns. But it will make the API much more complicated.
   
   The entire workflow can be
   1. The initial logical plan is still `DeleteFromTable`
   2. An optimizer rule turns `DeleteFromTable` into `GroupBasedDeleteFromTable`, which extracts translatable v2 `Filter`s from the catalyst condition and calls `getAffectedGroups` to get the initial group list. Then create v2 `Scan` with the initial group list.
   3. `GroupBasedDeleteFromTable` is defined as
   ```
   GroupBasedDeleteFromTable(
     SubqueryExpression(Distinct(Project(the_group_id_col, Filter(cond, V2Scan)))),
     Filter(Not(cond), V2Scan),
     V2Write)
   ```
   4. the planner rule turns `GroupBasedDeleteFromTable` into its corresponding physical plan
   5. The physical plan get the final group list from the subquery, create `BatchWrite` and write data, and finally commit the transaction.
   
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildOperationTable(
+      table: SupportsRowLevelOperations,
+      command: Command,
+      options: CaseInsensitiveStringMap): RowLevelOperationTable = {
+    val info = RowLevelOperationInfoImpl(command, options)
+    val builder = table.newRowLevelOperationBuilder(info)
+    val operation = builder.build()

Review comment:
       Nit: This doesn't need to assign builder to a variable.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   @rdblue @cloud-fan, I assumed the delete condition (not negated) would be explicitly passed to both scan builders by Spark. For instance, if the delete condition is `part_col = 'a' and id =1`, Spark would push it to the main scan builder and then provide an extra predicate on the filter attributes (e.g. `_file_name IN (...)`). Since the scan condition will be the same, data sources may cache and reuse some information between the scans.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       We can capture the actual DELETE/UPDATE/MERGE condition in `ReplaceData` and use that for job planning but the other concern about converting Catalyst expressions into data source filters without optimizing them remains.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       > Next, the V2ScanRelationPushDown will just work and apply operator pushdown to GroupBasedReplaceData.query. 
   
   I am afraid it is vice versa. The regular planning in `V2ScanRelationPushDown` can be applied to the `filter` subquery, not the `query` plan that contains remaining rows. The `query` plan contains a negated condition that cannot be pushed. That means the main scan must be fully built before `V2ScanRelationPushDown` kicks in. We cannot partially preconfigure the scan builder. Parquet pushdown can be enabled in the `filter` subquery, not the main scan.
   
   > this rule can also evaluate the query and collect the affected groups
   
   If we execute the filter subquery in an optimizer rule to form an IN filter while building the main scan, we may not be able to benefit from reusing subqueries and other optimization as it will no longer be one plan. That’s why leveraging the existing framework for runtime filtering seemed so promising.
   
   Based on what we discussed, I removed nested commands by adding more information to `ReplaceData`. I think it simplifies the implementation a little bit. In the future, `ReplaceData` may also contain a filter subquery that will be assigned in the rewrite plan (just like you mentioned). However, let’s put the runtime filtering out of scope for now. We know we have multiple ways to do it: either evaluate a subquery in an optimizer rule or leverage the runtime filtering framework for DS V2. We want to support a custom scan builder anyway. I can work on that immediately after this PR.
   
   What do you think, @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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       `ReplaceData` is `NestedCommand` Here is an example how it is handled.
   
   ```
   == Parsed Logical Plan ==
   'DeleteFromTable ('id <= 1)
   +- 'UnresolvedRelation [cat, ns1, test_table], [], false
   
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Optimized Logical Plan ==
   ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table, org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- Project [id#88, dep#89]
      +- Sort [_partition#91 ASC NULLS FIRST], false
         +- RepartitionByExpression [_partition#91], 5
            +- Filter NOT ((id#88 <= 1) <=> true)
               +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   
   == Physical Plan ==
   ReplaceData org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTable$PartitionBasedOperation$$anon$2$$anon$3@bc5bbcd
   +- AdaptiveSparkPlan isFinalPlan=false
      +- Project [id#88, dep#89]
         +- Sort [_partition#91 ASC NULLS FIRST], false, 0
            +- Exchange hashpartitioning(_partition#91, 5), REPARTITION_BY_NUM, [id=#182]
               +- Project [id#88, dep#89, _partition#91]
                  +- Filter NOT ((id#88 <= 1) <=> true)
                     +- BatchScan[id#88, dep#89, _partition#91] class org.apache.spark.sql.connector.catalog.InMemoryTable$InMemoryBatchScan RuntimeFilters: []
   
   ```
   
   Originally, `ReplaceData` is nested in `DeleteFromTable`. We need to execute that plan only if the table does not support DELETEs with filters.  Currently, `ReplaceData` becomes a top-level node in the optimizer but I will try to move that to the physical planning (i.e. `DataSourceV2Strategy`).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/SupportsRowLevelOperations.java
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.connector.catalog;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.write.RowLevelOperationBuilder;
+import org.apache.spark.sql.connector.write.RowLevelOperation;
+import org.apache.spark.sql.connector.write.RowLevelOperationInfo;
+
+/**
+ * A mix-in interface for {@link Table} row-level operations support. Data sources can implement
+ * this interface to indicate they support rewriting data for DELETE, UPDATE, MERGE operations.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface SupportsRowLevelOperations extends Table {
+  /**
+   * Returns a {@link RowLevelOperationBuilder} to build a {@link RowLevelOperation}.
+   * Spark will call this method while planning DELETE, UPDATE and MERGE operations
+   * that require rewriting data.
+   *
+   * @param info the row-level operation info such command (e.g. DELETE) and options

Review comment:
       Fixed.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.mutable
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId, V2ExpressionUtils}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.write.{RowLevelOperation, RowLevelOperationInfoImpl, RowLevelOperationTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
+
+  protected def buildRowLevelOperation(
+      table: SupportsRowLevelOperations,
+      command: Command): RowLevelOperation = {
+    val info = RowLevelOperationInfoImpl(command, CaseInsensitiveStringMap.empty())
+    val builder = table.newRowLevelOperationBuilder(info)
+    builder.build()
+  }

Review comment:
       Spark only supports SQL DELETE statements and there is no way for a user to pass options, unfortunately. However, the data source APIs assume there will be options in the future. They just happen to be empty right now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTableSuite.scala
##########
@@ -0,0 +1,524 @@
+/*
+ * 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.connector
+
+import java.util
+import java.util.Collections
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.sql.{AnalysisException, DataFrame, Encoders, QueryTest, Row}
+import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryRowLevelOperationTableCatalog}
+import org.apache.spark.sql.connector.expressions.LogicalExpressions._
+import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.sql.types.StructType
+
+abstract class DeleteFromTableSuiteBase
+  extends QueryTest with SharedSparkSession with BeforeAndAfter with AdaptiveSparkPlanHelper {
+
+  import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+  import testImplicits._
+
+  before {
+    spark.conf.set("spark.sql.catalog.cat", classOf[InMemoryRowLevelOperationTableCatalog].getName)
+  }
+
+  after {
+    spark.sessionState.catalogManager.reset()
+    spark.sessionState.conf.unsetConf("spark.sql.catalog.cat")
+  }
+
+  private val namespace = Array("ns1")
+  private val ident = Identifier.of(namespace, "test_table")
+  private val tableNameAsString = "cat." + ident.toString
+
+  protected def extraTableProps(): util.Map[String, String] = {
+    Collections.emptyMap[String, String]
+  }
+
+  private def catalog: InMemoryRowLevelOperationTableCatalog = {
+    val catalog = spark.sessionState.catalogManager.catalog("cat")
+    catalog.asTableCatalog.asInstanceOf[InMemoryRowLevelOperationTableCatalog]
+  }
+
+  test("EXPLAIN only delete") {
+    createAndInitTable("id INT, dep STRING", """{ "id": 1, "dep": "hr" }""")
+
+    sql(s"EXPLAIN DELETE FROM $tableNameAsString WHERE id <= 10")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Nil)
+  }
+
+  test("delete from empty tables") {
+    createTable("id INT, dep STRING")
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with basic filters") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id <= 1")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(3, "hr") :: Nil)
+  }
+
+  test("delete with aliases") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": 3, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString AS t WHERE t.id <= 1 OR t.dep = 'hr'")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Row(2, "software") :: Nil)
+  }
+
+  test("delete with IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id IN (1, null)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, "software") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with NOT IN predicates") {
+    createAndInitTable("id INT, dep STRING",
+      """{ "id": 1, "dep": "hr" }
+        |{ "id": 2, "dep": "software" }
+        |{ "id": null, "dep": "hr" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (null, 1)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(2, "software") :: Row(null, "hr") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE id NOT IN (1, 10)")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(1, "hr") :: Row(null, "hr") :: Nil)
+  }
+
+  test("delete with conditions on nested columns") {
+    createAndInitTable("id INT, complex STRUCT<c1:INT,c2:STRING>, dep STRING",
+      """{ "id": 1, "complex": { "c1": 3, "c2": "v1" }, "dep": "hr" }
+        |{ "id": 2, "complex": { "c1": 2, "c2": "v2" }, "dep": "software" }
+        |""".stripMargin)
+
+    sql(s"DELETE FROM $tableNameAsString WHERE complex.c1 = id + 2")
+
+    checkAnswer(
+      sql(s"SELECT * FROM $tableNameAsString"),
+      Row(2, Row(2, "v2"), "software") :: Nil)
+
+    sql(s"DELETE FROM $tableNameAsString t WHERE t.complex.c1 = id")
+
+    checkAnswer(sql(s"SELECT * FROM $tableNameAsString"), Nil)
+  }
+
+  test("delete with IN subqueries") {
+    withTempView("deleted_id", "deleted_dep") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedIdDF = Seq(Some(0), Some(1), None).toDF()
+      deletedIdDF.createOrReplaceTempView("deleted_id")
+
+      val deletedDepDF = Seq("software", "hr").toDF()
+      deletedDepDF.createOrReplaceTempView("deleted_dep")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT * FROM deleted_id)
+           | AND
+           | dep IN (SELECT * FROM deleted_dep)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": -1, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hardware") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IS NULL
+           | OR
+           | id IN (SELECT value + 2 FROM deleted_id)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Nil)
+
+      append("id INT, dep STRING",
+        """{ "id": null, "dep": "hr" }
+          |{ "id": 2, "dep": "hr" }
+          |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(2, "hr") :: Row(null, "hr") :: Nil)
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | id IN (SELECT value + 2 FROM deleted_id)
+           | AND
+           | dep = 'hr'
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(-1, "hr") :: Row(1, "hr") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with multi-column IN subqueries") {
+    withTempView("deleted_employee") {
+      createAndInitTable("id INT, dep STRING",
+        """{ "id": 1, "dep": "hr" }
+          |{ "id": 2, "dep": "hardware" }
+          |{ "id": null, "dep": "hr" }
+          |""".stripMargin)
+
+      val deletedEmployeeDF = Seq((None, "hr"), (Some(1), "hr")).toDF()
+      deletedEmployeeDF.createOrReplaceTempView("deleted_employee")
+
+      sql(
+        s"""DELETE FROM $tableNameAsString
+           |WHERE
+           | (id, dep) IN (SELECT * FROM deleted_employee)
+           |""".stripMargin)
+
+      checkAnswer(
+        sql(s"SELECT * FROM $tableNameAsString"),
+        Row(2, "hardware") :: Row(null, "hr") :: Nil)
+    }
+  }
+
+  test("delete with NOT IN subqueries") {

Review comment:
       Thanks, @kazuyukitanimura! Will rebase this PR now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       We actually have to prevent that (added the new rule to the list of rules that cannot be excluded).
   
   Here is how a DELETE command may look like.
   
   ```
   == Analyzed Logical Plan ==
   DeleteFromTable (id#88 <= 1)
   :- RelationV2[id#88, dep#89] cat.ns1.test_table
   +- ReplaceData RelationV2[id#88, dep#89] cat.ns1.test_table
      +- Filter NOT ((id#88 <= 1) <=> true)
         +- RelationV2[id#88, dep#89, _partition#91] cat.ns1.test_table
   ```
   
   The condition we should push down to the source is the DELETE condition `(id < 1)` (not the condition in the filter on top of the scan). Suppose we have a data source that can replace files. We have two files: File A contains IDs 1, 2, 3 and File B contains IDs 5, 6, 7. If we want to delete the record with ID = 1, we should push down the actual delete condition (ID = 1) for correct file pruning. Once the data source determines that only File A contains records to delete, we need to read the entire file and determine what records did not match the condition (that's what that filter on top of the scan is doing). Those records (IDs 2, 3 in our example) have to be written back to the data source as it can only replace files.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Not yet. Once we agree this is the way to go, I'll add a test suite.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala
##########
@@ -255,7 +255,13 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat
     case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _, _, Some(write)) =>
       OverwritePartitionsDynamicExec(planLater(query), refreshCache(r), write) :: Nil
 
-    case DeleteFromTable(relation, condition) =>
+    case DeleteFromTableWithFilters(r: DataSourceV2Relation, filters) =>
+      DeleteFromTableExec(r.table.asDeletable, filters.toArray, refreshCache(r)) :: Nil
+
+    case DeleteFromTable(_, _, Some(rewritePlan)) =>
+      planLater(rewritePlan) :: Nil
+
+    case DeleteFromTable(relation, condition, None) =>

Review comment:
       That's a good question. Technically, `OptimizeMetadataOnlyDeleteFromTable` has almost identical logic but applies only to plans that reference `ReplaceData` and does not throw any exceptions. Another substantial difference is the new rule does the check in the optimizer while the current check is done during physical planning. 
   
   I am looking for feedback on this one.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       This optimizer rule contains logic similar to what we have in `DataSourceV2Strategy`. However, it is done in the optimizer to avoid building `Scan` and `Write` if a DELETE operation can be handled using filters.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildReadRelation(relation, operationTable, metadataAttrs)
+
+    // construct a plan that contains unmatched rows in matched groups that must be carried over
+    // such rows do not match the condition but have to be copied over as the source can replace
+    // only groups of rows (e.g. if a source supports replacing files, unmatched rows in matched
+    // files must be carried over)

Review comment:
       It may be helpful to have a note here about how the file pruning happens and that it is safe to negate the condition because filter pushdown is handled specially in `RowLevelCommandScanRelationPushDown`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala
##########
@@ -55,3 +55,9 @@ trait AnalysisOnlyCommand extends Command {
   // on the `AnalysisContext`
   def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan
 }
+
+/**
+ * A command that is nested within another command after the analysis and does not have to be
+ * executed eagerly. Such commands will be either removed or made top-level in the optimizer.
+ */
+trait NestedCommand extends Command

Review comment:
       Resolving this thread not to mislead anyone with outdated info.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
aokolnychyi edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1062176426


   > I'm afraid a unified physical plan execution may fit one very well but is awkward for the other.
   
   @cloud-fan, I am not proposing a unified execution plan. In fact, we should rewrite delta-based and group-based rewrites in a completely different way, just like you said. Whatever I have here is for group-based sources. I had an early prototype for delta-based sources in the old PR #33008. It has a separate execution path with `WriteDelta` instead of `ReplaceData`.
   
   > Figure out which "groups" are affected w.r.t. the DELETE/UPDATE/MERGE condition. Since the condition may contain subqueries, it's better to let Spark submit the job to evaluate the condition and let data source report the "groups", so that Spark can figure out which "groups" are affected. The execution path can be: Spark asks the data source to give an initial list of "groups" w.r.t. the condition, and then create a data source scan with the initial "groups". The data source scan has a hidden column to report the "group id", and spark runs the scan with the post-scan filters to collect the group ids.
   Scan these "groups" to get the new data after DELETE/UPDATE/MERGE, and write new data to new "groups".
   Commit the transaction to remove old "groups" and add new "groups".
   
   My proposal should support the use case you describe. Here is a quote from the design doc:
   
   > For example, sources that support replacing individual files may use static filters to prune partitions and files using metadata. This will allow them to find files that may have rows to delete/update. Since the metadata filtering is not precise and rewriting data is expensive, it may make sense to scan the potentially matching files for matches using a separate filtering query to reduce the amount of data to rewrite.
   
   I am proposing to leverage runtime filtering for this. It would just require a new optimizer rule and would work well with the proposed APIs in this PR. We can even include runtime filtering in 3.3 as DS V2 supports runtime filtering. I mentioned this in the SPIP doc. Here is how that optimizer rule can look like (it will assign a subquery to find matching groups).
   
   ```
   object RowLevelCommandDynamicPruning extends Rule[LogicalPlan] with PredicateHelper {
   
     override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
       // apply special dynamic filtering only for plans that don't support deltas
       case RewrittenRowLevelCommand(
           command: RowLevelCommand,
           DataSourceV2ScanRelation(_, scan: SupportsRuntimeFiltering, _),
           rewritePlan: ReplaceData) if conf.dynamicPartitionPruningEnabled && isCandidate(command) =>
   
         // use reference equality to find exactly the required scan relations
         val newRewritePlan = rewritePlan transformUp {
           case r: DataSourceV2ScanRelation if r.scan eq scan =>
             val pruningKeys = V2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
             val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
             Filter(dynamicPruningCond, r)
         }
         command.withNewRewritePlan(newRewritePlan)
     }
   }
   ```
   
   Sources like Delta can expose a metadata column `_file` and support dynamic filtering on it during row-level operations. I think that will be cleaner than having a notion of a group in the APIs. This will also produce a much cleaner plan. We won't have two separate jobs.
   
   Tests I added also show how writes can access what "groups" were scanned and should be replaced. We don't have to pass replaced groups explicitly. For instance, `PartitionBasedOperation` added for testing remembers the scan that was used and the write has access to it. That's why scans can report to writes what groups must be replaced.
   
   The purpose of `RowLevelOperation` is to coordinate scans and writes.
   
   I think the proposed API should work well for Delta, Iceberg, and even Hive (where groups are partitions).


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yea, we are talking about the same algorithm, just different ways to do interaction between Spark and data sources:
   1. Let the data source implicitly obtain the affected "groups" within itself, making the API simpler. (The current proposal)
   2. Ask the data source to report "group id" so that Spark can calculate the affected groups by itself. (My new proposal)
   
   The above query plan you provided makes sense, the problem is how to build that subquery filter (the join) without any DS v2 API for the data source to report the "group id"?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
##########
@@ -37,7 +37,11 @@ class SparkOptimizer(
 
   override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
     // TODO: move SchemaPruning into catalyst
-    SchemaPruning :: V2ScanRelationPushDown :: V2Writes :: PruneFileSourcePartitions :: Nil
+    SchemaPruning :: RowLevelCommandScanRelationPushDown :: V2ScanRelationPushDown :: V2Writes ::

Review comment:
       After looking at this more, I agree with this direction. There is no need to overcomplicate either case.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       It is also extremely easy to build the filtering subquery. In case of DELETE, we just need to find matches so we have to take the delete condition and evaluate it against the target table. Spark will take care of rewriting that condition as joins using existing rules.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +501,52 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Expression
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Expression,

Review comment:
       Looks good to me too.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data sources must produce all rows in a group
+   * if any are returned. Some sources will avoid pushing filters into files (file granularity),
+   * while others will avoid pruning files within a partition (partition granularity).
+   * <p>
+   * For example, if a source can only replace partitions, all rows from a partition must
+   * be returned by the scan, even if a filter can narrow the set of changes to a single file
+   * in the partition. Similarly, a source that can swap individual files must produce all rows
+   * of files where at least one record must be changed, not just the rows that must be changed.
+   */
+  ScanBuilder newScanBuilder(CaseInsensitiveStringMap options);

Review comment:
       Wait, the opposite. We need special scans for group-based sources but delta-based sources can use regular scans.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1073744276


   The query plan LGTM. I have the same question as @rdblue . Now we always push the delete condition as a predicate to the main `Scan`, and your proposal will add a new optimizer rule which will push the delete condition to the "runtime filter" `Scan`, then use its result to build an IN predicate and push it to the main `Scan`. In this case, I think we don't need to push the static predicate to the main `Scan` at the beginning.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
##########
@@ -388,3 +391,62 @@ object ExtractSingleColumnNullAwareAntiJoin extends JoinSelectionHelper with Pre
     case _ => None
   }
 }
+
+/**
+ * An extractor for operations such as DELETE and MERGE that require rewriting data.
+ *
+ * This class extracts the following entities:
+ *  - the row-level command (such as [[DeleteFromTable]]);
+ *  - the read relation in the rewrite plan that can be either [[DataSourceV2Relation]] or
+ *  [[DataSourceV2ScanRelation]] depending on whether the planning has already happened;
+ *  - the current rewrite plan.

Review comment:
       I kept the minimum required logic for group-based deletes for now. You are right, this extractor will change to support UPDATE and delta-based sources. What about updating the description once we make those changes? For now, there will be exactly one read relation.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       I mention the proposed APIs for delta-based operations in the design [doc](https://docs.google.com/document/d/12Ywmc47j3l2WF4anG5vL4qlrhT2OKigb7_EbIKhxg60). We will need new APIs but they will extend the existing ones. They won't be entirely different.
   
   ```
   interface SupportsDelta extends RowLevelOperation {
     DeltaWriteBuilder newWriteBuilder(LogicalWriteInfo info);
     NamedReference[] rowId();
   }
   
   interface DeltaWriteBuilder extends WriteBuilder {
    DeltaWrite build();
   }
   
   interface DeltaWrite extends Write {
     DeltaBatchWrite toBatch();
   }
   
   interface DeltaBatchWrite extends BatchWrite {
     DeltaWriterFactory createBatchWriterFactory(...);
   }
   
   interface DeltaWriterFactory extends WriterFactory {
     DeltaWriter<InternalRow> createWriter(long partId, long taskId);
   }
   ```




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       @cloud-fan, `DeleteFromTableWithFilters` is an optimization for `SupportsRowLevelOperations`. Existing deletes with filters would be unaffected. That being said, I am going to combine the existing logic in `DataSourceV2Strategy` with the optimizer rule I added, like discussed [here](https://github.com/apache/spark/pull/35395#discussion_r805292918). That way, we will have the filter conversion logic just in one place. Let me know if you agree with that.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       I'll handle that separately. No need to assign a rewrite plan.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       I tried to keep the API consistent by following what we did for scans and writes. I think we have mix-in traits to pass extra information to builders (e.g. predicate pushdown). In this case, the produced row-level operation tells Spark whether deltas are supported and what's the row ID. We can't pass the row ID from Spark. For now, we always pass a SQL command and options to builders. My worry is that we may have valid use cases for mix-in traits on the builder for passing extra information in the future.
   
   I am happy to discuss, though.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala
##########
@@ -284,6 +284,22 @@ case class OverwritePartitionsDynamicExec(
     copy(query = newChild)
 }
 
+/**
+ * Physical plan node to replace data in existing tables.
+ */
+case class ReplaceDataExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    write: Write) extends V2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet

Review comment:
       I think we can remove it from this exec node. We will need it for the delta-based plan that I'll add later.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))

Review comment:
       How about `d.copy(rewritePlan = Some(rewritePlan))`? I'm a little concerned that we replace `aliasedTable` with `r` here.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.

Review comment:
       I added `DeleteFromTableWithFilters` and we call `SupportsDelete$canDeleteWhere` only once now.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       The import looks a bit weird. I can do an aliased import if that's any better.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -176,6 +176,52 @@ object OverwritePartitionsDynamic {
   }
 }
 
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommand with NestedCommand {

Review comment:
       Added.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       Cause we need to push down the command condition, I couldn't use the existing rule. If anyone has any ideas on how to avoid a separate rule, I'll be glad to do that.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildReadRelation(relation, operationTable, metadataAttrs)

Review comment:
       It isn't very clear what needs to happen to construct a read relation, when there is already a relation for the table. I think a fix is to make the method name describe what is happening more closely, like `relationWithRequiredAttrs`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       What about `cond = Some(TrueLiteral)` and `TruncatableTable`?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      val partitionColumnRef = FieldReference(PartitionKeyColumn.name)

Review comment:
       If this were a constant, then it could be reused to build the required distribution.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala
##########
@@ -446,17 +493,49 @@ object DescribeColumn {
   def getOutputAttrs: Seq[Attribute] = DescribeCommandSchema.describeColumnAttributes()
 }
 
+trait RowLevelCommand extends Command with SupportsSubquery {
+  def condition: Option[Expression]
+  def rewritePlan: Option[LogicalPlan]
+  def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand
+}
+
 /**
  * The logical plan of the DELETE FROM command.
  */
 case class DeleteFromTable(
     table: LogicalPlan,
-    condition: Option[Expression]) extends UnaryCommand with SupportsSubquery {
-  override def child: LogicalPlan = table
-  override protected def withNewChildInternal(newChild: LogicalPlan): DeleteFromTable =
-    copy(table = newChild)
+    condition: Option[Expression],
+    rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand {
+
+  override def children: Seq[LogicalPlan] = if (rewritePlan.isDefined) {
+    table :: rewritePlan.get :: Nil
+  } else {
+    table :: Nil
+  }
+
+  override def withNewRewritePlan(newRewritePlan: LogicalPlan): RowLevelCommand = {
+    copy(rewritePlan = Some(newRewritePlan))
+  }
+
+  override protected def withNewChildrenInternal(
+      newChildren: IndexedSeq[LogicalPlan]): DeleteFromTable = {
+    if (newChildren.size == 1) {
+      copy(table = newChildren.head, rewritePlan = None)
+    } else {
+      require(newChildren.size == 2, "DeleteFromTable expects either one or two children")
+      val Seq(newTable, newRewritePlan) = newChildren.take(2)
+      copy(table = newTable, rewritePlan = Some(newRewritePlan))
+    }
+  }
 }
 
+/**
+ * The logical plan of the DELETE FROM command that can be executed using data source filters.
+ */
+case class DeleteFromTableWithFilters(
+    table: LogicalPlan,
+    condition: Seq[sources.Filter]) extends LeafCommand

Review comment:
       I would probably move `DeleteFromTableWithFilters` to a follow-up commit since it is an optimization and not needed for correctness.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperationBuilder.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+
+/**
+ * An interface for building a {@link RowLevelOperation}.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperationBuilder {

Review comment:
       are we going to add mix-in traits for this interface?




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)

Review comment:
       I took another look at `V2ScanRelationPushDown`. I think we can make filter pushdown work there by adding separate branches for `RewrittenRowLevelCommand` but it does not seem to help. Instead, it would make the existing rule even more complicated. Apart from that, we also can't apply regular logic for aggregate pushdown as we have to look at the condition in the row-level operation. Essentially, we have to make sure that none of the logic in the existing rule work for row-level operations. At this point, I agree keeping a separate rule seems cleaner.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       Switched to `QueryCompilationErrors`.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   I've rebased this PR and went through the code and comments one more time. This PR should be ready for another review round.
   
   cc @huaxingao @dongjoon-hyun @sunchao @cloud-fan @viirya @rdblue


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule will still
+ * assign a rewrite plan but the optimizer will check whether this particular DELETE statement can
+ * be handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw QueryCompilationErrors.tableDoesNotSupportDeletesError(t)
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs that may be used for grouping data on write
+    // for instance, JDBC data source may cluster data by shard/host before writing
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)

Review comment:
       I'm a little confused here. Why do we need to include metadata attributes to build the "main scan" which is the input of the write operation? According to our previous discussion, metadata attributes should be used to build the scan for collecting the affected groups.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 edited a comment on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35395:
URL: https://github.com/apache/spark/pull/35395#issuecomment-1061514337


   Sorry for the late review as I've been struggling with the API design for a long time. I understand that we want to have a unified DS v2 API for delta-based and group-based data sources, but these two have very different execution paths and I'm afraid a unified physical plan execution may fit one very well but is awkward for the other.
   
   The execution path of group-based data sources (please correct me if my understanding is wrong):
   1. Figure out which "groups" are affected w.r.t. the DELETE/UPDATE/MERGE condition. Since the condition may contain subqueries, it's better to let Spark submit the job to evaluate the condition and let data source report the "groups", so that Spark can figure out which "groups" are affected. The execution path can be: Spark asks the data source to give an initial list of "groups" w.r.t. the condition, and then create a data source scan with the initial "groups". The data source scan has a hidden column to report the "group id", and spark runs the scan with the post-scan filters to collect the group ids.
   2. Scan these "groups" to get the new data after DELETE/UPDATE/MERGE, and write new data to new "groups".
   3. Commit the transaction to remove old "groups" and add new "groups".
   
   To fit this execution path, the API proposal from me is (naming is not finalized)
   ```
   interface SupportsGroupBasedUpsert extends Table {
     String[] getAffectedGroups(Filter[] condition);
     // This scan should put an extra string column at the end of each row to indicate the group id.
     ScanBuilder newScanBuilder(String[] groups, CaseInsensitiveStringMap options);
    // The created `Write` must exends `SupportReplaceGroups`
     WriteBuilder newWriteBuilder(CaseInsensitiveStringMap options);
   }
   ...
   interface SupportReplaceGroups extends Write {
     BatchWrite toBatch(String[] groups);
   }
   ```
   Note: the API above assumes the group can always be represented as a string. If there are objections, we can use binary and add an API to serialize/deserialize groups. e.g.
   ```
   Group[] getAffectedGroups(Filter[] condition);
   Group fromBinary(byte[] binary);
   // This scan should put an extra binary column at the end of each row to indicate the group id.
   ScanBuilder newScanBuilder(Group[] groups, CaseInsensitiveStringMap options);
   ...
   interface Group extends Serializable {}
   ```
   We can also follow `ContinuousPartitionReader` if we want more type-safety and don't want to use hidden columns. But it will make the API much more complicated.
   
   The entire workflow can be
   1. The initial logical plan is still `DeleteFromTable`
   2. An optimizer rule turns `DeleteFromTable` into `GroupBasedDeleteFromTable`, which extracts translatable v2 `Filter`s from the catalyst condition and calls `getAffectedGroups` to get the initial group list. Then create v2 `Scan` with the initial group list.
   3. `GroupBasedDeleteFromTable` is defined as
   ```
   GroupBasedDeleteFromTable(
     SubqueryExpression(Distinct(Project(the_group_id_col, Filter(cond, V2Scan)))),
     Filter(Not(cond), V2Scan),
     V2Write)
   ```
   4. the planner rule turns `GroupBasedDeleteFromTable` into its corresponding physical plan
   5. The physical plan get the final group list from the subquery, create `BatchWrite` and write data, and finally commit the transaction.
   
   We can also update the rule `V2Writes` to add shuffle/sort for `GroupBasedDeleteFromTable`


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I'm thinking about it a bit more. I think it's clearer if we resolve group-based DELETE in two steps:
   1. create a plan to get affected groups
   2. create a plan to get the undeleted rows
   
   These two steps can share one `ScanBuilder` instance.
   
   First, we should have an analyzer rule to rewrite `DeleteFromTable`, similar to what this PR does. But we can make it simpler. We should just replace `DeleteFromTable` with a new plan
   ```
   case class GroupBasedReplaceData(
       table: NamedRelation,
       condition: Expression,
       affectedGroups: LogicalPlan,
       query: LogicalPlan,
       write: Option[Write] = None) extends V2WriteCommand with SupportsSubquery {
   }
   ```
   - The first 2 arguments are directly from `DeleteFromTable`, so that we can go back to `DeleteFromTable` later.
   - The `affectedGroups` is just a `Filter` with negated DELETE condition, on a DS v2 relation with `RowLevelOperationTable` that is introduced in this PR.
   - The `query` is almost the same as `affectedGroups` except that its `Filter` condition is the original DELETE condition.
   - The `affectedGroups` and `query` share the same `RowLevelOperationTable` instance, and `RowLevelOperationTable` should create `ScanBuilder` only once, e.g.
   ```
   private lazy val scanBuilder = table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty())
   override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = scanBuilder
   ```
   
   Second, we apply a new optimizer rule right before `V2ScanRelationPushDown`. This new rule does predicate pushdown for the `affectedGroups`, to configure `RowLevelOperationTable.scanBuilder`. In the future, when we add APIs to create dedicated v2 `Scan` to get affected groups, this rule can also evaluate the query and collect the affected groups. This rule can also go back to `DeleteFromTable` if we can.
   
   Next, the `V2ScanRelationPushDown` will just work and apply operator pushdown to `GroupBasedReplaceData.query`. Note that, predicate pushdown will be done again here, which is useful: the first predicate pushdown allows the data source to determine which groups to replace, the second predicate pushdown allows the data source to do pruning at runtime (like parquet row group pruning).
   
   Finally, we evaluate `GroupBasedReplaceDataExec`, similar to what this PR does.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       I'm thinking about it a bit more. I think it's clearer if we resolve group-based DELETE in two steps:
   1. create a plan to get affected groups
   2. create a plan to get the undeleted rows
   These two steps can share one `ScanBuilder` instance.
   
   First, we should have an analyzer rule to rewrite `DeleteFromTable`, similar to what this PR does. But we can make it simpler. We should just replace `DeleteFromTable` with a new plan
   ```
   case class GroupBasedReplaceData(
       table: NamedRelation,
       condition: Expression,
       affectedGroups: LogicalPlan,
       query: LogicalPlan,
       write: Option[Write] = None) extends V2WriteCommand with SupportsSubquery {
   }
   ```
   - The first 2 arguments are directly from `DeleteFromTable`, so that we can go back to `DeleteFromTable` later.
   - The `affectedGroups` is just a `Filter` with negated DELETE condition, on a DS v2 relation with `RowLevelOperationTable` that is introduced in this PR.
   - The `query` is almost the same as `affectedGroups` except that its `Filter` condition is the original DELETE condition.
   - The `affectedGroups` and `query` share the same `RowLevelOperationTable` instance, and `RowLevelOperationTable` should create `ScanBuilder` only once, e.g.
   ```
   private lazy val scanBuilder = table.asReadable.newScanBuilder(CaseInsensitiveStringMap.empty())
   override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = scanBuilder
   ```
   
   Second, we apply a new optimizer rule right before `V2ScanRelationPushDown`. This new rule does predicate pushdown for the `affectedGroups`, to configure `RowLevelOperationTable.scanBuilder`. In the future, when we add APIs to create dedicated v2 `Scan` to get affected groups, this rule can also evaluate the query and collect the affected groups. This rule can also go back to `DeleteFromTable` if we can.
   
   Next, the `V2ScanRelationPushDown` will just work and apply operator pushdown to `GroupBasedReplaceData.query`. Note that, predicate pushdown will be done again here, which is useful: the first predicate pushdown allows the data source to determine which groups to replace, the second predicate pushdown allows the data source to do pruning at runtime (like parquet row group pruning).
   
   Finally, we evaluate `GroupBasedReplaceDataExec`, similar to what this PR does.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/OptimizeMetadataOnlyDeleteFromTable.scala
##########
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, DeleteFromTableWithFilters, LogicalPlan}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, TruncatableTable}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources
+
+/**
+ * Checks whether a delete using filters is possible and nullifies the rewrite plan
+ * if the source can handle this delete without executing the rewrite plan.
+ *
+ * Note this rule must be run after expression optimization but before scan planning.
+ */
+object OptimizeMetadataOnlyDeleteFromTable extends Rule[LogicalPlan] with PredicateHelper {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+    case d @ DeleteFromTable(relation: DataSourceV2Relation, cond, Some(_)) =>

Review comment:
       We also may not be able to get rid of the optional rewrite plan in `DeleteFromTable`. This node holds the actual delete condition that is pushed down during job planning in the optimizer. We cannot push down the condition in the rewrite plan as it has been negated for sources that replace groups. In other words, job planning for row-level commands is special and we cannot get rid of `DeleteFromTable` in the analyzer as it provides the required context to plan the job (hence a special rule for job planning).




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       I don't think the result will be discarded. Let me explain how I see this:
   
   - The data source plans input splits for groups that may have matches using parts of the condition that can be converted into data source filters. 
   - The data source scan can cache group IDs or input splits that may have matches and expose filter attributes for runtime filtering. In case of Delta, the scan needs to remember a set of files that potentially have matches.
   - If the data source supports runtime filtering, Spark will assign a filtering subquery that will be executed at runtime based on the SQL command. The same scan object will be used in the filtering subquery and in the row-level operation. In case of Delta, the runtime filtering will be done using the `_file_name` metadata column.
   - Spark executes the filtering subquery via the existing runtime filtering mechanism, collects unique values for the filtering attributes and passes them into the scan. In case of Delta, the scan will receive a set of files that have matches.
   - The data source uses the passed unique values to filter out cached group IDs or tasks and reports back input splits for groups that definitely have matches. In case of Delta, it can filter out previously cached files and include tasks only for files that have matches.
   - Writes have access to scans, so each scan can report a list of affected group IDs (which was cached). It can be in any format: strings, case classes, etc. Whatever a particular data source needs.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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 #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryRowLevelOperationTable.scala
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.connector.catalog
+
+import java.util
+
+import org.apache.spark.sql.connector.distributions.{Distribution, Distributions}
+import org.apache.spark.sql.connector.expressions.{FieldReference, LogicalExpressions, NamedReference, SortDirection, SortOrder, Transform}
+import org.apache.spark.sql.connector.read.{Scan, ScanBuilder}
+import org.apache.spark.sql.connector.write.{BatchWrite, LogicalWriteInfo, RequiresDistributionAndOrdering, RowLevelOperation, RowLevelOperationBuilder, RowLevelOperationInfo, Write, WriteBuilder, WriterCommitMessage}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+class InMemoryRowLevelOperationTable(
+    name: String,
+    schema: StructType,
+    partitioning: Array[Transform],
+    properties: util.Map[String, String])
+  extends InMemoryTable(name, schema, partitioning, properties) with SupportsRowLevelOperations {
+
+  override def newRowLevelOperationBuilder(
+      info: RowLevelOperationInfo): RowLevelOperationBuilder = {
+    () => PartitionBasedOperation(info.command)
+  }
+
+  case class PartitionBasedOperation(command: Command) extends RowLevelOperation {
+    private final val PARTITION_COLUMN_REF = FieldReference(PartitionKeyColumn.name)
+
+    var configuredScan: InMemoryBatchScan = _
+
+    override def requiredMetadataAttributes(): Array[NamedReference] = {
+      Array(PARTITION_COLUMN_REF)
+    }
+
+    override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = {
+      new InMemoryScanBuilder(schema) {
+        override def build: Scan = {
+          val scan = super.build()
+          configuredScan = scan.asInstanceOf[InMemoryBatchScan]
+          scan
+        }
+      }
+    }
+
+    override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = new WriteBuilder {
+
+      override def build(): Write = new Write with RequiresDistributionAndOrdering {
+        override def requiredDistribution(): Distribution = {
+          Distributions.clustered(Array(PARTITION_COLUMN_REF))
+        }
+
+        override def requiredOrdering(): Array[SortOrder] = {
+          Array[SortOrder](
+            LogicalExpressions.sort(
+              PARTITION_COLUMN_REF,
+              SortDirection.ASCENDING,
+              SortDirection.ASCENDING.defaultNullOrdering())
+          )
+        }
+
+        override def toBatch: BatchWrite = PartitionBasedReplaceData(configuredScan)
+
+        override def description(): String = "InMemoryWrite"
+      }
+    }
+
+    override def description(): String = "InMemoryPartitionReplaceOperation"
+  }
+
+  private case class PartitionBasedReplaceData(scan: InMemoryBatchScan) extends TestBatchWrite {
+
+    override def commit(messages: Array[WriterCommitMessage]): Unit = dataMap.synchronized {
+      val newData = messages.map(_.asInstanceOf[BufferedRows])
+      val readRows = scan.data.flatMap(_.asInstanceOf[BufferedRows].rows)

Review comment:
       Yea we agree on the algorithm, the question is how the affected "groups" are determined and passed around. BTW, Delta does want to support conditions with subqueries, and we want to make sure the newly proposed API here can work with subqueries: https://github.com/delta-io/delta/issues/826
   
   > Spark executes the filtering subquery via the existing runtime filtering mechanism, collects unique values for the filtering attributes ...
   
   This is the key problem. How does Spark know how to get the file name column? There is no DS v2 API for Delta to tell Spark: hey you can select `_file_name` column to get the "group id". Are we going to add an implicit assumption that, the `Scan` will add an extra column silently like my proposal? If yes, I think this solves the problem and is actually very similar to my proposal with the difference that we use the existing hidden column and runtime filter API to report "group id" from the source and use it to determine the final affected "groups".




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] viirya commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   cc @sunchao @dongjoon-hyun @huaxingao to double-check. 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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Not}
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations, TruncatableTable}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case DataSourceV2Relation(_: TruncatableTable, _, _, _, _) if cond == TrueLiteral =>
+          // don't assign a rewrite plan as the table supports truncation
+          d
+
+        case r @ DataSourceV2Relation(t: SupportsRowLevelOperations, _, _, _, _) =>
+          val table = buildOperationTable(t, DELETE, CaseInsensitiveStringMap.empty())
+          val rewritePlan = buildReplaceDataPlan(r, table, cond)
+          // keep the original relation in DELETE to also attempt deleting with filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")

Review comment:
       Oops, will fix.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   @cloud-fan, thanks for thinking this through with me! I genuinely want to make sure this API works for every data source.
   
   The two scenarios you mention seem correct to me. I agree data sources may or may not want Spark to run an extra query to filter out groups. If a data source is OK with simple static filter pushdown, Spark should not trigger runtime filtering.
   
   > Now let's look at the proposed API. I think it works pretty well for the simplified process, and I believe you have a plan to support the non-simple process by adding extension APIs. Just to make sure it will be backward compatible and we are on the same page
   
   I propose to handle the non-simple case by just adding an optimizer rule that would work with the APIs added in this PR. No extra classes are necessary. I think we can achieve that by reusing the same `Scan` object in the main and runtime filtering queries. Let me show how that will work.
   
   For example, data sources implement `RowLevelOperation` that will give access to a scan builder.
   
   ```
   ScanBuilder newScanBuilder(CaseInsensitiveStringMap options);
   ```
   
   Spark will pass a static predicate to the scan builder. That is sufficient for the simple use case. The more complicated use case requires Spark to do runtime filtering on top of this. That's why if `Scan` implements `SupportsRuntimeFiltering`, Spark can execute that extra query via its usual runtime filtering mechanism. The idea is to reuse the same `Scan` object we built for the row-level operation in the filtering subquery. That way, we avoid planning the same job twice and runtime filtering becomes a totally optional step for which data sources opt in by returning a filterable `Scan` by group ID. We may not need a separate scan builder for the filtering subquery.
   
   Does this make sense?
   
   > row-level operation does not even call out the data writing behavior. Shall we consider other names like SupportsDataUpdating, SupportsUPSERT, et.?
   
   I went back and forth on the name and then settled on `SupportsRowLevelOperations` as it describes all there SQL operations: DELETE, UPDATE, MERGE. I am open to alternatives, though. I think `SupportsDataUpdating` focuses mostly on UPDATE and it is not very descriptive for DELETE and MERGE. Also, UPSERT has slightly different meaning compared to DELETE, UPDATE and MERGE so `SupportsUPSERT` can be misinterpreted. I'd love Spark to support UPSERT but it is beyond the scope of my proposal.
   
   > Why do we put the replace plan as a parameter of DeleteFromTable? Can we simply replace DeleteFromTable with the new replace plan in the rule?
   
   It is done so that we can attempt a metadata-only delete via `SupportsDelete`. Suppose our DELETE condition has predicates only on partition columns. Hive, Delta, Iceberg will be able to handle such a delete using a metadata operation without rewriting data. The rewrite plan is assigned in the analyzer but we check whether a metadata delete is possible only in the optimizer. Until then, we don't know whether the rewrite plan will be executed.
   


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/java/org/apache/spark/sql/connector/write/RowLevelOperation.java
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.connector.write;
+
+import org.apache.spark.annotation.Experimental;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+/**
+ * A logical representation of a data source DELETE, UPDATE, or MERGE operation that requires
+ * rewriting data.
+ *
+ * @since 3.3.0
+ */
+@Experimental
+public interface RowLevelOperation {
+
+  /**
+   * The SQL row-level operation.
+   */
+  enum Command {
+    DELETE, UPDATE, MERGE
+  }
+
+  /**
+   * Returns the description associated with this row-level operation.
+   */
+  default String description() {
+    return this.getClass().toString();
+  }
+
+  /**
+   * Returns the actual SQL operation being performed.
+   */
+  Command command();
+
+  /**
+   * Returns a {@link ScanBuilder} to configure a {@link Scan} for this row-level operation.
+   * <p>
+   * Sources fall into two categories: those that can handle a delta of rows and those that need
+   * to replace groups (e.g. partitions, files). Sources that handle deltas allow Spark to quickly
+   * discard unchanged rows and have no requirements for input scans. Sources that replace groups
+   * of rows can discard deleted rows but need to keep unchanged rows to be passed back into
+   * the source. This means that scans for such data sources must produce all rows in a group
+   * if any are returned. Some sources will avoid pushing filters into files (file granularity),
+   * while others will avoid pruning files within a partition (partition granularity).
+   * <p>
+   * For example, if a source can only replace partitions, all rows from a partition must
+   * be returned by the scan, even if a filter can narrow the set of changes to a single file
+   * in the partition. Similarly, a source that can swap individual files must produce all rows
+   * of files where at least one record must be changed, not just the rows that must be changed.
+   */
+  ScanBuilder newScanBuilder(CaseInsensitiveStringMap options);

Review comment:
       Yep. Scans for delta-based operations will be no different from regular scans.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.v2
+
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression}
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.expressions.filter.{Filter => V2Filter}
+import org.apache.spark.sql.connector.read.ScanBuilder
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.sources.Filter
+
+/**
+ * A rule that builds a scan in row-level operations that require rewriting data.
+ *
+ * Note this rule must be run before [[V2ScanRelationPushDown]] as row-level operations must be
+ * planned in a special way.
+ */
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import DataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    // push down the filter from the command condition instead of the filter in the rewrite plan,
+    // which may be negated for sources that only support replacing groups of data (e.g. files)
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>
+      val table = relation.table.asRowLevelOperationTable
+      val condition = command.condition
+      val scanBuilder = table.newScanBuilder(relation.options)
+
+      val (pushedFilters, remainingFilters) = pushFilters(condition, relation.output, scanBuilder)
+      val pushedFiltersStr = if (pushedFilters.isLeft) {
+        pushedFilters.left.get.mkString(", ")
+      } else {
+        pushedFilters.right.get.mkString(", ")
+      }
+
+      val (scan, output) = PushDownUtils.pruneColumns(scanBuilder, relation, relation.output, Nil)
+
+      logInfo(
+        s"""
+           |Pushing operators to ${relation.name}
+           |Pushed filters: ${pushedFiltersStr}
+           |Filters that were not pushed: ${remainingFilters.mkString(",")}

Review comment:
       Will do.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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


   @viirya, it seems easier to catch some nits with smaller PRs. For instance, I just updated Javadoc in a few places while reading it again. Easy to miss some spots if the PR is large. I'd probably vote for smaller PRs if that's fine with everybody.


-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildReadRelation(relation, operationTable, metadataAttrs)
+
+    // construct a plan that contains unmatched rows in matched groups that must be carried over
+    // such rows do not match the condition but have to be copied over as the source can replace
+    // only groups of rows (e.g. if a source supports replacing files, unmatched rows in matched
+    // files must be carried over)

Review comment:
       Added.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, operationTable.operation)
+
+    // construct a read relation and include all required metadata columns
+    val readRelation = buildReadRelation(relation, operationTable, metadataAttrs)

Review comment:
       Renamed.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,84 @@
+/*
+ * 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 org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{EqualNullSafe, Expression, Literal, Not}
+import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, Filter, LogicalPlan, ReplaceData}
+import org.apache.spark.sql.connector.catalog.{SupportsDelete, SupportsRowLevelOperations}
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements [[SupportsDelete]] and [[SupportsRowLevelOperations]], this rule assigns
+ * a rewrite plan but the optimizer will check whether this particular DELETE statement can be
+ * handled by simply passing delete filters to the connector. If yes, the optimizer will then
+ * discard the rewrite plan.
+ */
+object RewriteDeleteFromTable extends RewriteRowLevelCommand {
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case d @ DeleteFromTable(aliasedTable, cond, None) if d.resolved =>
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) =>
+          val operation = buildRowLevelOperation(tbl, DELETE)
+          val table = RowLevelOperationTable(tbl, operation)
+          val deleteCond = cond.getOrElse(Literal.TrueLiteral)
+          val rewritePlan = buildReplaceDataPlan(r, table, deleteCond)
+          // keep the original relation in DELETE to attempt deleting using filters
+          DeleteFromTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>
+          // don't assign a rewrite plan as the table supports deletes only with filters
+          d
+
+        case DataSourceV2Relation(t, _, _, _, _) =>
+          throw new AnalysisException(s"Table $t does not support DELETE statements")
+
+        case _ =>
+          d
+      }
+  }
+
+  // build a rewrite plan for sources that support replacing groups of data (e.g. files, partitions)
+  private def buildReplaceDataPlan(
+      relation: DataSourceV2Relation,
+      operationTable: RowLevelOperationTable,
+      cond: Expression): ReplaceData = {
+
+    // resolve all required metadata attrs (e.g. may be needed for grouping data on write)

Review comment:
       Done.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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] aokolnychyi commented on a change in pull request #35395: [SPARK-38085][SQL] DataSource V2: Handle DELETE commands for group-based sources

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



##########
File path: sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala
##########
@@ -37,7 +37,11 @@ class SparkOptimizer(
 
   override def earlyScanPushDownRules: Seq[Rule[LogicalPlan]] =
     // TODO: move SchemaPruning into catalyst
-    SchemaPruning :: V2ScanRelationPushDown :: V2Writes :: PruneFileSourcePartitions :: Nil
+    SchemaPruning :: RowLevelCommandScanRelationPushDown :: V2ScanRelationPushDown :: V2Writes ::

Review comment:
       Keeping them separate for now as `V2ScanRelationPushDown` is already complicated and none of that logic applies.




-- 
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.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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