You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/12/03 19:46:14 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request #3661: Spark: Implement copy-on-write DELETE

aokolnychyi opened a new pull request #3661:
URL: https://github.com/apache/iceberg/pull/3661


   This PR adds support for copy-on-write DELETE commands using the new connector APIs.
   
   Open tasks:
   - Detect when a child query contains a shuffle and insert a round-robin partitioning in `ExtendedV2Writes`.
   - Spark currently does not support any aggregations inside dynamic filters with adaptive execution enabled, which means we cannot use scalar subqueries with adaptive execution (see the corresponding test modification and comment).


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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExprId
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationInfoImpl
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import scala.collection.mutable
+
+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 dedupAttrs(attrs: Seq[AttributeReference]): Seq[AttributeReference] = {
+    val exprIds = mutable.Set.empty[ExprId]
+    attrs.flatMap { attr =>
+      if (exprIds.contains(attr.exprId)) {
+        None
+      } else {
+        exprIds += attr.exprId
+        Some(attr)
+      }
+    }
+  }
+
+  protected def resolveRequiredMetadataAttrs(
+      relation: DataSourceV2Relation,
+      operation: RowLevelOperation): Seq[AttributeReference] = {
+
+    ExtendedV2ExpressionUtils.resolveRefs[AttributeReference](
+      operation.requiredMetadataAttributes,
+      relation)
+  }
+
+  protected def resolveRowIdAttrs(

Review comment:
       Missed to remove.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>

Review comment:
       I'll remove.

##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>
+        Join(relation, m.sourceTable, LeftSemi, Some(cond), JoinHint.NONE)
+
+      case u: UpdateTable =>
+        // UPDATEs with subqueries may be rewritten using a UNION with two identical scan relations
+        // each scan relation will get its own dynamic filter that will be shared during execution
+        // the analyzer will assign different expr IDs for each scan relation output attributes
+        // that's why the condition may refer to invalid attr expr IDs and must be transformed
+        val attrMap = AttributeMap(u.table.output.zip(relation.output))
+        val transformedCond = cond transform {
+          case attr: AttributeReference if attrMap.contains(attr) => attrMap(attr)
+        }
+        Filter(transformedCond, relation)
+
+      case _ =>
+        Filter(cond, relation)

Review comment:
       Good call.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       Yeah, I do have a projection in `ExtendedV2Writes`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(
+      table: Table,
+      plan: LogicalPlan,
+      allowScanDuplication: Boolean): Option[LogicalPlan] = {
+
+    val scanRelations = plan.collect {
+      case r: DataSourceV2Relation if r.table eq table => r
+      case r: DataSourceV2ScanRelation if r.relation.table eq table => r
+    }
+
+    // in some cases, the optimizer replaces the v2 scan relation with a local relation
+    // for example, there is no reason to query the table if the condition is always false
+    // that's why it is valid not to find the corresponding v2 scan relation
+
+    scanRelations match {
+      case relations if relations.isEmpty =>
+        None
+
+      case Seq(relation) =>
+        Some(relation)
+
+      case Seq(relation1: DataSourceV2Relation, relation2: DataSourceV2Relation)
+        if allowScanDuplication && (relation1.table eq relation2.table) =>
+        Some(relation1)
+
+      case Seq(relation1: DataSourceV2ScanRelation, relation2: DataSourceV2ScanRelation)
+        if allowScanDuplication && (relation1.scan eq relation2.scan) =>
+        Some(relation1)
+
+      case Seq(relation1, relation2) if allowScanDuplication =>
+        throw new AnalysisException(s"Row-level scan relations don't match: $relation1, $relation2")
+
+      case relations if allowScanDuplication =>
+        throw new AnalysisException(s"Expected 2 row-level scan relations: $relations")

Review comment:
       It is a little hard to explain the logic here without seeing the UPDATE code. I'll remove it for now and then we can discuss it again for UPDATEs.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       Can you give me a little more context on 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(
+      table: Table,
+      plan: LogicalPlan,
+      allowScanDuplication: Boolean): Option[LogicalPlan] = {
+
+    val scanRelations = plan.collect {
+      case r: DataSourceV2Relation if r.table eq table => r
+      case r: DataSourceV2ScanRelation if r.relation.table eq table => r
+    }
+
+    // in some cases, the optimizer replaces the v2 scan relation with a local relation
+    // for example, there is no reason to query the table if the condition is always false
+    // that's why it is valid not to find the corresponding v2 scan relation
+
+    scanRelations match {
+      case relations if relations.isEmpty =>
+        None
+
+      case Seq(relation) =>
+        Some(relation)
+
+      case Seq(relation1: DataSourceV2Relation, relation2: DataSourceV2Relation)
+        if allowScanDuplication && (relation1.table eq relation2.table) =>
+        Some(relation1)
+
+      case Seq(relation1: DataSourceV2ScanRelation, relation2: DataSourceV2ScanRelation)
+        if allowScanDuplication && (relation1.scan eq relation2.scan) =>
+        Some(relation1)
+
+      case Seq(relation1, relation2) if allowScanDuplication =>
+        throw new AnalysisException(s"Row-level scan relations don't match: $relation1, $relation2")
+
+      case relations if allowScanDuplication =>
+        throw new AnalysisException(s"Expected 2 row-level scan relations: $relations")

Review comment:
       What is the benefit of failing if you thought there may be 2? What about when users have turned off dynamic filtering or something that would have created the second 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {

Review comment:
       I think it would be more clear if `ReplaceData.table` were called `relation` 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on pull request #3661: Spark: Implement copy-on-write DELETE

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


   cc @rdblue @RussellSpitzer @szehon-ho @flyrain @karuppayya @szehon-ho @kbendick 


-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING)
+    };
+
+    Distribution expectedDistribution = Distributions.ordered(expectedOrdering);
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeletePartitionedUnsortedTable() {

Review comment:
       Does everybody agree 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Coalesce
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.CASE_WHEN
+import org.apache.spark.sql.catalyst.trees.TreePattern.IF
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule similar to SimplifyConditionalsInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] {

Review comment:
       Is it possible to let the original rules simplify the parsed `DeleteFromTable` and only convert to `DeleteFromIcebergTable` after this has run? Something like this:
   
   ```scala
         parsedPlan match {
           case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), Some(condition)) if isSimplified(cond) =>
             DeleteFromIcebergTable(aliasedTable, condition)
           case _ =>
             parsedPlan
         }
   
     private isSimplified(cond: Expression): Boolean = {
       !_.containsAnyPattern(CASE_WHEN, IF, NULL_LITERAL, TRUE_OR_FALSE_LITERAL, INSET)
     }
   ```




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -103,7 +114,43 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
     if (isIcebergCommand(sqlTextAfterSubstitution)) {
       parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan]
     } else {
-      delegate.parsePlan(sqlText)
+      val parsedPlan = delegate.parsePlan(sqlText)
+      parsedPlan match {
+        case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), condition) =>
+          DeleteFromIcebergTable(aliasedTable, condition)
+        case _ =>
+          parsedPlan
+      }
+    }
+  }
+
+  object UnresolvedIcebergTable {
+
+    def unapply(plan: LogicalPlan): Option[LogicalPlan] = {
+      EliminateSubqueryAliases(plan) match {
+        case UnresolvedRelation(multipartIdentifier, _, _) if isIcebergTable(multipartIdentifier) =>
+          Some(plan)
+        case _ =>
+          None
+      }
+    }
+
+    private def isIcebergTable(multipartIdent: Seq[String]): Boolean = {
+      val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(SparkSession.active, multipartIdent.asJava)
+      catalogAndIdentifier.catalog match {
+        case tableCatalog: TableCatalog =>
+          Try(tableCatalog.loadTable(catalogAndIdentifier.identifier))
+            .map(isIcebergTable)
+            .getOrElse(false)
+
+        case _ =>
+          false
+      }
+    }
+
+    private def isIcebergTable(table: Table): Boolean = table match {
+      case _: SparkTable =>true

Review comment:
       Nit: missing space before `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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {

Review comment:
       I can rename. I called it `table` as it extended `V2WriteCommand` before. Looks like all v2 write commands call this a table.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       This node has custom resolution as the input query may have more columns than the table. For example, below the query contains `id#249, dep#250, _file#254, _pos#255L`.
   
   ```
   +- ReplaceData RelationV2[id#249, dep#250] testhive.default.table
      +- Filter NOT ((id#249 IN (list#248 []) AND (dep#250 = hr)) <=> true)
         :  +- Project [(value#19 + 2) AS (value + 2)#251]
         :     +- SubqueryAlias deleted_id
         :        +- View (`deleted_id`, [value#19])
         :           +- LocalRelation [value#19]
         +- RelationV2[id#249, dep#250, _file#254, _pos#255L] testhive.default.table
   ```
   
   I thought I was projecting away the metadata columns in the writer but I am no longer sure. I'll have to check.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Coalesce
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.CASE_WHEN
+import org.apache.spark.sql.catalyst.trees.TreePattern.IF
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule similar to SimplifyConditionalsInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] {

Review comment:
       We could probably construct a fake plan with a local relation and a filter and run it through the optimizer but I am not sure it is 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)

Review comment:
       We can't apply the regular resolution as described [here](https://github.com/apache/iceberg/pull/3661/files#r767936935). Spark uses `query.schema` for regular writes. It feels more natural to use the incoming query types to construct a row schema. I think it also gives us more precise results (e.g. actual nullability). 




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      // range distribution only makes sense if the sort order is set
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.sortOrder().isUnsorted()) {
+        return HASH;
+      } else {
+        return deleteMode;
+      }
+    } else {
+      // use hash distribution if write distribution is range or hash

Review comment:
       One reason is to avoid changing the behavior we have right now. The second reason is performance. I think it is pretty nice that we can do a hash partitioning by file as it is way more efficient than a range-based shuffle.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Is there a better name? It wasn't clear to me what "pruning" meant. What about `RowLevelCommandDynamicPruning`?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -103,7 +114,43 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
     if (isIcebergCommand(sqlTextAfterSubstitution)) {
       parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan]
     } else {
-      delegate.parsePlan(sqlText)
+      val parsedPlan = delegate.parsePlan(sqlText)
+      parsedPlan match {
+        case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), condition) =>
+          DeleteFromIcebergTable(aliasedTable, condition)
+        case _ =>
+          parsedPlan
+      }
+    }
+  }
+
+  object UnresolvedIcebergTable {
+
+    def unapply(plan: LogicalPlan): Option[LogicalPlan] = {
+      EliminateSubqueryAliases(plan) match {
+        case UnresolvedRelation(multipartIdentifier, _, _) if isIcebergTable(multipartIdentifier) =>
+          Some(plan)
+        case _ =>
+          None
+      }
+    }
+
+    private def isIcebergTable(multipartIdent: Seq[String]): Boolean = {
+      val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(SparkSession.active, multipartIdent.asJava)
+      catalogAndIdentifier.catalog match {
+        case tableCatalog: TableCatalog =>
+          Try(tableCatalog.loadTable(catalogAndIdentifier.identifier))
+            .map(isIcebergTable)
+            .getOrElse(false)
+
+        case _ =>
+          false
+      }
+    }
+
+    private def isIcebergTable(table: Table): Boolean = table match {
+      case _: SparkTable =>true

Review comment:
       Removed.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
##########
@@ -52,6 +56,7 @@ public static void startMetastoreAndSpark() {
         .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
         .config("spark.sql.shuffle.partitions", "4")
         .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+        .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))

Review comment:
       I think it'll be fine, it's just a bit weird. How long does this suite take to run? Is it something that needs to be run with all 3 catalog types, or can we get away with just running it with one or 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/V2WriteCommandLike.scala
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+
+// a node similar to V2WriteCommand in Spark but does not extend Command
+// as ReplaceData and WriteDelta that extend this trait are nested within other commands
+trait V2WriteCommandLike extends UnaryNode {

Review comment:
       Ideas/thoughts are welcome.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala
##########
@@ -103,7 +114,43 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI
     if (isIcebergCommand(sqlTextAfterSubstitution)) {
       parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan]
     } else {
-      delegate.parsePlan(sqlText)
+      val parsedPlan = delegate.parsePlan(sqlText)
+      parsedPlan match {
+        case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), condition) =>

Review comment:
       This is a trick I had to do as we now rewrite in the analyzer. Moreover, existing analyzer rules in Spark break our logic (e.g. column resolution for updates). That's why we parse using Spark and then replace immediately with a custom node.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -555,10 +560,13 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException {
 
     createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT());
 
-    sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", tableName);
-    assertEquals("Should have expected rows",
-        ImmutableList.of(row(2, "hardware"), row(null, "hr")),
-        sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName));
+    // TODO: Spark does not support AQE and DPP with aggregates at the moment

Review comment:
       That's an issue I don't have a solution for. It will be pretty tough if even possible to overcome.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java
##########
@@ -209,15 +210,19 @@ private int batchSize(boolean isParquetOnly, boolean isOrcOnly) {
 
   @Override
   public Statistics estimateStatistics() {
+    return estimateStatistics(table.currentSnapshot());
+  }
+
+  protected Statistics estimateStatistics(Snapshot snapshot) {
     // its a fresh table, no data
-    if (table.currentSnapshot() == null) {
+    if (snapshot == null) {
       return new Stats(0L, 0L);
     }
 
     // estimate stats using snapshot summary only for partitioned tables (metadata tables are unpartitioned)
     if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) {
       LOG.debug("using table metadata to estimate table statistics");
-      long totalRecords = PropertyUtil.propertyAsLong(table.currentSnapshot().summary(),
+      long totalRecords = PropertyUtil.propertyAsLong(snapshot.summary(),

Review comment:
       It is related. I just forgot to invoke it from `SparkCopyOnWriteScan`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       Yeah, the order matters. I'll add a comment. It is a bit of a hack and we are also lucky dynamic rules are applied after pre-CBO.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) extends Rule[LogicalPlan] with PredicateHelper {

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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       In other words, we can discard the metadata columns only after the write has been built in the optimizer. That's why I added a projection in `ExtendedV2Writes`. Since the rewrite happens in the analyzer,  `ReplaceData` must be considered resolved as long as the data columns are aligned. That's why this node has custom resolution.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
##########
@@ -52,6 +56,7 @@ public static void startMetastoreAndSpark() {
         .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
         .config("spark.sql.shuffle.partitions", "4")
         .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+        .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))

Review comment:
       I don't think that having a random that can cause test failures is a great idea. I'd rather set it in a few tests that need 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {

Review comment:
       This seems fine to me.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       I've spent some time thinking and it does not look like there is a guarantee an extra round-robin repartition would improve the performance. In some cases, it may even make things worse. Here are my thoughts.
   
   - With round-robin, we won’t have to evaluate a condition or join predicate twice. In MERGE, we won’t have to evaluate the join condition and won’t have to merge the rows. (GOOD)
   - With round-robin, we won't read the data twice from the storage. If needed, we fetch shuffle data. (GOOD)
   - With round-robin, we will have an extra shuffle (both write and read). Shuffles are expensive and tend to behave poorly at scale. Also, the data will be scattered all over the place so each reduce task will need to read from most nodes. (BAD)
   - With round-robin, we have extra complexity that may not be always beneficial for the query performance. In some cases, this may degrade the performance or use more resources. (BAD)
   
   At this point, I am inclined to skip inserting an extra round-robin repartition as even merging the rows should not be that bad considering that we operate on `InternalRow`. I've spent so much time tuning shuffles recently so that I am a bit skeptical it will help 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       I mean we should probably check if the query contains a shuffle if the write asks for range partitioning. Recomputing the query may be expensive during the skew estimation step.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
##########
@@ -52,6 +56,7 @@ public static void startMetastoreAndSpark() {
         .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
         .config("spark.sql.shuffle.partitions", "4")
         .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+        .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))

Review comment:
       We test with 3 catalogs and 3 different file formats so the overhead seems reasonable.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)
+
+    // 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
+    val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       This seems odd to me. If there is no condition, why add one and continue the rewrite? We know that `true` can be done in metadata, so we could check this above and avoid building the plan entirely.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)

Review comment:
       Is the purpose of overriding `dataInput` just to keep the original row schema?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.In
+import org.apache.spark.sql.catalyst.expressions.InSet
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.INSET
+import org.apache.spark.sql.catalyst.trees.TreePattern.NULL_LITERAL
+import org.apache.spark.sql.catalyst.trees.TreePattern.TRUE_OR_FALSE_LITERAL
+import org.apache.spark.sql.types.BooleanType
+import org.apache.spark.util.Utils
+
+/**
+ * A rule similar to ReplaceNullWithFalseInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] {

Review comment:
       Ideally, we would just cover deletes in the original rule but we cannot do that. Our tests fail without 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING)
+    };
+
+    Distribution expectedDistribution = Distributions.ordered(expectedOrdering);
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeletePartitionedUnsortedTable() {

Review comment:
       Why not distribute by the partition 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -178,8 +178,25 @@ public DistributionMode distributionMode() {
     }
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.spec().isUnpartitioned() && table.sortOrder().isUnsorted()) {

Review comment:
       This is okay as it is. Let's not block on it. There are good arguments for using HASH.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -164,9 +161,45 @@ public Scan build() {
         spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
   }
 
-  public Scan buildMergeScan() {
-    return new SparkMergeScan(
-        spark, table, readConf, ignoreResiduals,
-        schemaWithMetadataColumns(), filterExpressions);
+  public Scan buildCopyOnWriteScan() {

Review comment:
       I moved the construction of `TableScan` here. I'll do the same for regular scans in the next PR.
   Merge-on-read will use the existing scan for batch queries (with runtime filtering and everything).




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {

Review comment:
       If the strategy is copy-on-write, then it makes sense to me to actually write the replacement files using `range` 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)

Review comment:
       Yeah, we replace the original table with `RowLevelOperationTable`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)
+
+    // 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
+    val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       Well, it makes a little bit more sense in a generic case. I did not do any major adaptations specific to Iceberg. It is pretty cheap to assign a rewrite plan and then it gets discarded 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      // range distribution only makes sense if the sort order is set
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.sortOrder().isUnsorted()) {
+        return HASH;
+      } else {
+        return deleteMode;
+      }
+    } else {
+      // use hash distribution if write distribution is range or hash

Review comment:
       One reason is to avoid changing the behavior we have right now. The second reason is performance. I think it is pretty nice that we can do a hash partitioning by file as it is way more efficient than a range-based shuffle (in most cases).




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       This is probably something we will have to address before it is merged.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       A rule that injects dynamic filters that are handled by the existing framework for v2 tables.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,79 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.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.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.types.StructType
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Scan planning changed so drastically before the 3.2 release so that it was easier to put this into a separate 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -611,11 +619,22 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte
 
     // append thread
     Future<?> appendFuture = executorService.submit(() -> {
+      List<Integer> ids = ImmutableList.of(1, 2);
+      Dataset<Row> inputDF = spark.createDataset(ids, Encoders.INT())
+          .withColumnRenamed("value", "id")
+          .withColumn("dep", lit("hr"));
+
       for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) {
         while (barrier.get() < numOperations * 2) {
           sleep(10);
         }
-        sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName);

Review comment:
       I had to modify as these became metadata deletes.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>

Review comment:
       Removed.

##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>
+        Join(relation, m.sourceTable, LeftSemi, Some(cond), JoinHint.NONE)
+
+      case u: UpdateTable =>
+        // UPDATEs with subqueries may be rewritten using a UNION with two identical scan relations
+        // each scan relation will get its own dynamic filter that will be shared during execution
+        // the analyzer will assign different expr IDs for each scan relation output attributes
+        // that's why the condition may refer to invalid attr expr IDs and must be transformed
+        val attrMap = AttributeMap(u.table.output.zip(relation.output))
+        val transformedCond = cond transform {
+          case attr: AttributeReference if attrMap.contains(attr) => attrMap(attr)
+        }
+        Filter(transformedCond, relation)
+
+      case _ =>
+        Filter(cond, relation)

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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       The projection is inserted only after the write is built as the extra columns can be part of the distribution and ordering requested by the write. That happens in the optimizer. Before that, we need to mark this as fully resolved.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java
##########
@@ -209,15 +210,19 @@ private int batchSize(boolean isParquetOnly, boolean isOrcOnly) {
 
   @Override
   public Statistics estimateStatistics() {
+    return estimateStatistics(table.currentSnapshot());
+  }
+
+  protected Statistics estimateStatistics(Snapshot snapshot) {
     // its a fresh table, no data
-    if (table.currentSnapshot() == null) {
+    if (snapshot == null) {
       return new Stats(0L, 0L);
     }
 
     // estimate stats using snapshot summary only for partitioned tables (metadata tables are unpartitioned)
     if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) {
       LOG.debug("using table metadata to estimate table statistics");
-      long totalRecords = PropertyUtil.propertyAsLong(table.currentSnapshot().summary(),
+      long totalRecords = PropertyUtil.propertyAsLong(snapshot.summary(),

Review comment:
       Should have the missing part 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")

Review comment:
       Merge-on-read will come in following PRs.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {

Review comment:
       I am not a big fan of this part so ideas are welcome.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Coalesce
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.CASE_WHEN
+import org.apache.spark.sql.catalyst.trees.TreePattern.IF
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule similar to SimplifyConditionalsInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] {

Review comment:
       Same story like with `ExtendedReplaceNullWithFalseInPredicate`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>

Review comment:
       Are these other cases needed for `DeleteFromIcebergTable`? We should know that it is a `SupportsRowLevelOperations` because it is an Iceberg table 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved

Review comment:
       This and `outputResolved` look like they are from the original `V2WriteCommand`. Why did you add them here instead of to `V2WriteCommandLike`?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(

Review comment:
       I meant that the original relation in row-level commands is kind of split into "write" and "scan" relations for the same table. Then we read from the scan relation and write into the write relation.
   
   I am still wondering what would be the best naming here. I want to refer to the relation we read from.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)

Review comment:
       Why not use `r.output`, the write relation's schema? That's what we do for normal writes isn't it? Should we just apply the `ResolveOutputRelation` 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.scala
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {
+    val tableAttrNames = table.output.map(_.name)
+    query.output.filter(attr => tableAttrNames.exists(conf.resolver(_, attr.name)))
+  }
+
+  override def outputResolved: Boolean = {
+    assert(table.resolved && query.resolved,
+      "`outputResolved` can only be called when `table` and `query` are both resolved.")
+
+    // take into account only incoming data columns and ignore metadata columns in the query

Review comment:
       Added a comment on resolution, @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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       Added a comment.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -178,8 +178,25 @@ public DistributionMode distributionMode() {
     }
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.spec().isUnpartitioned() && table.sortOrder().isUnsorted()) {

Review comment:
       One reason I did not consider RANGE by `_file` and `_pos` is because distributing by `_file` on S3 can break the existing distribution and ordering and create more files as we potentially break partition boundaries.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -178,8 +178,25 @@ public DistributionMode distributionMode() {
     }
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.spec().isUnpartitioned() && table.sortOrder().isUnsorted()) {

Review comment:
       If the delete mode was specifically set to RANGE, I think we should probably pass it through. I see the logic of not doing that if we're going to use no custom sort order, but I think we should sort by `_file` and `_pos` if there is no order to keep rows in the same files and original order.
   
   If we're using `_file` and `_pos`, then RANGE would mean to rebalance records across files. That makes some sense if records were already mostly in sorted order. HASH would maintain existing file boundaries. The question for me is whether we should assume that RANGE indicates that it's okay to change file boundaries... If we had a total ordering before and deleted large chunks of records, this would make sense. But if the files/records weren't already a total ordering then it could mess up column stats. Whether this assumption holds is not clear, so I would say let's defer to the user's explicit request, since this is the delete mode.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);

Review comment:
       Why doesn't this use hash distribution by file and order by file and pos?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
##########
@@ -52,6 +56,7 @@ public static void startMetastoreAndSpark() {
         .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
         .config("spark.sql.shuffle.partitions", "4")
         .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+        .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))

Review comment:
       Technically, adaptive execution should have no impact. That's why it would be hard to come up with particular tests to set this. If this suite wasn't so expensive, I'd add two separate runs.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #3661: Spark: Implement copy-on-write DELETE

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #3661:
URL: https://github.com/apache/iceberg/pull/3661


   


-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Coalesce
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.CASE_WHEN
+import org.apache.spark.sql.catalyst.trees.TreePattern.IF
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule similar to SimplifyConditionalsInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] {

Review comment:
       Ah, nevermind. I see now that the conversion to Iceberg plan actually comes immediately after parsing.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       Do you mean contains a sort?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>

Review comment:
       Do we need non-Delete cases 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchScan.java
##########
@@ -209,15 +210,19 @@ private int batchSize(boolean isParquetOnly, boolean isOrcOnly) {
 
   @Override
   public Statistics estimateStatistics() {
+    return estimateStatistics(table.currentSnapshot());
+  }
+
+  protected Statistics estimateStatistics(Snapshot snapshot) {
     // its a fresh table, no data
-    if (table.currentSnapshot() == null) {
+    if (snapshot == null) {
       return new Stats(0L, 0L);
     }
 
     // estimate stats using snapshot summary only for partitioned tables (metadata tables are unpartitioned)
     if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) {
       LOG.debug("using table metadata to estimate table statistics");
-      long totalRecords = PropertyUtil.propertyAsLong(table.currentSnapshot().summary(),
+      long totalRecords = PropertyUtil.propertyAsLong(snapshot.summary(),

Review comment:
       Is this an unrelated 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() {

Review comment:
       This looks good.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(

Review comment:
       One option can be to call them "read" and "write" relations to avoid using "scan".




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExprId
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationInfoImpl
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import scala.collection.mutable
+
+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 dedupAttrs(attrs: Seq[AttributeReference]): Seq[AttributeReference] = {
+    val exprIds = mutable.Set.empty[ExprId]
+    attrs.flatMap { attr =>
+      if (exprIds.contains(attr.exprId)) {
+        None
+      } else {
+        exprIds += attr.exprId
+        Some(attr)
+      }
+    }
+  }
+
+  protected def resolveRequiredMetadataAttrs(
+      relation: DataSourceV2Relation,
+      operation: RowLevelOperation): Seq[AttributeReference] = {
+
+    ExtendedV2ExpressionUtils.resolveRefs[AttributeReference](
+      operation.requiredMetadataAttributes,
+      relation)
+  }
+
+  protected def resolveRowIdAttrs(

Review comment:
       Nit: this isn't needed yet since deltas aren't supported.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -555,10 +560,13 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException {
 
     createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT());
 
-    sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", tableName);
-    assertEquals("Should have expected rows",
-        ImmutableList.of(row(2, "hardware"), row(null, "hr")),
-        sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName));
+    // TODO: Spark does not support AQE and DPP with aggregates at the moment

Review comment:
       That's an issue I don't have a solution for.
   It will be pretty tough (if even possible) to overcome (without modifying 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       Yeah, the order matters. I'll add a comment.
   It is a bit of a hack and we are also lucky dynamic rules are applied after pre-CBO.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       Yeah, the order matters. I'll add a comment.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -75,6 +76,22 @@ public void removeTables() {
     sql("DROP TABLE IF EXISTS deleted_dep");
   }
 
+  @Test
+  public void testDeleteWithFalseCondition() {
+    createAndInitUnpartitionedTable();
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName);
+
+    sql("DELETE FROM %s WHERE id = 1 AND id > 20", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots()));

Review comment:
       The optimizer is not that smart and actually keeps the condition. That's why we have a new snapshot.  It is a valid test, though, as we run though metrics checks.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) extends Rule[LogicalPlan] with PredicateHelper {

Review comment:
       Will rename.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       I'm inclined to leave it out. If we need it, we can add it later. It is also something that Spark users are already familiar with adding manually because there are many situations that require one. Just writing a clustered data set to a table by adding a global sort commonly requires manually adding a repartition.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)

Review comment:
       We can't apply the regular resolution as described [here](https://github.com/apache/iceberg/pull/3661/files#r767936935). Spark uses `query.schema` for regular writes.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);

Review comment:
       Why doesn't this use hash distribution by file and order by file and pos? Shouldn't that be the default if there is no property specified? I think we should only use `NONE` if it is specifically requested. Otherwise we should try to match the original distribution and ordering.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(

Review comment:
       I've updated to use "read".




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);

Review comment:
       Yeah, let's change it. I think that's better than not returning anything.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved

Review comment:
       I can move the `resolved` definition to the parent trait but the implementation of `outputResolved` will be different in each command and won't be the same as in `V2WriteCommand`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       If there is a projection before the replace data, then we don't need to modify 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -555,10 +560,13 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException {
 
     createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT());
 
-    sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", tableName);
-    assertEquals("Should have expected rows",
-        ImmutableList.of(row(2, "hardware"), row(null, "hr")),
-        sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName));
+    // TODO: Spark does not support AQE and DPP with aggregates at the moment

Review comment:
       It leads to a runtime exception.
   
   ```
   class org.apache.spark.sql.catalyst.plans.logical.Aggregate cannot be cast to class org.apache.spark.sql.execution.SparkPlan
   ```
   
   I did debug it at some point. If I remember correctly, it was related to the fact that dynamic subqueries in Spark cannot have other subqueries or aggregates.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>

Review comment:
       Removed.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)
+
+    // 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
+    val deleteCond = cond.getOrElse(Literal.TrueLiteral)

Review comment:
       Changed.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>

Review comment:
       We don't need necessarily need it. I did not remove it when copying from the Spark PR.

##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(r, cond, Some(rewritePlan))
+
+        case DataSourceV2Relation(_: SupportsDelete, _, _, _, _) =>

Review comment:
       We don't necessarily need it. I did not remove it when copying from the Spark 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,79 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.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.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.types.StructType
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>

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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      // range distribution only makes sense if the sort order is set
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.sortOrder().isUnsorted()) {
+        return HASH;
+      } else {
+        return deleteMode;
+      }
+    } else {
+      // use hash distribution if write distribution is range or hash

Review comment:
       Added a comment 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -164,9 +161,45 @@ public Scan build() {
         spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
   }
 
-  public Scan buildMergeScan() {
-    return new SparkMergeScan(
-        spark, table, readConf, ignoreResiduals,
-        schemaWithMetadataColumns(), filterExpressions);
+  public Scan buildCopyOnWriteScan() {

Review comment:
       I moved the construction of `TableScan` here. I'll do the same for regular scans in the next 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING)
+    };
+
+    Distribution expectedDistribution = Distributions.ordered(expectedOrdering);
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeletePartitionedUnsortedTable() {

Review comment:
       I'd probably default to hash in this case rather than 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)

Review comment:
       Do you need `table=table` in `copy`?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -555,10 +560,13 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException {
 
     createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT());
 
-    sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", tableName);
-    assertEquals("Should have expected rows",
-        ImmutableList.of(row(2, "hardware"), row(null, "hr")),
-        sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName));
+    // TODO: Spark does not support AQE and DPP with aggregates at the moment

Review comment:
       What happens when adaptive execution is not disabled?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -555,10 +560,13 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException {
 
     createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT());
 
-    sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", tableName);
-    assertEquals("Should have expected rows",
-        ImmutableList.of(row(2, "hardware"), row(null, "hr")),
-        sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", tableName));
+    // TODO: Spark does not support AQE and DPP with aggregates at the moment

Review comment:
       That's an issue I don't have a solution for. It will be pretty tough (if even possible) to overcome (without modifying 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       I've spent some time thinking and it does not look like there is a guarantee an extra round-robin repartition would improve the performance. In some cases, it may even make things worse. Here are my thoughts.
   
   - With round-robin, we won’t have to evaluate a condition or join predicate twice. In MERGE, we won’t have to evaluate the join condition and won’t have to merge the rows. (GOOD)
   - With round-robin, we won't read the data twice from the storage. If needed, we fetch shuffle data. (GOOD)
   - With round-robin, we will have an extra shuffle (both write and read). Shuffles are expensive and tend to behave poorly at scale. (BAD)
   - With round-robin, we have extra complexity that may not be always beneficial for the query performance. In some cases, this may degrade the performance or use more resources. (BAD)
   
   At this point, I am inclined to skip inserting an extra round-robin repartition as even merging the rows should not be that bad considering that we operate on `InternalRow`. I've spent so much time tuning shuffles recently so that I am a bit skeptical it will help 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      rewritePlan match {
+        case rd: ReplaceData =>
+          rd.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>

Review comment:
       It looks weird to pull table out here, when this was matched from `rd.table`. I think the rename would help, but this is minor.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteDeleteFromTable.scala
##########
@@ -0,0 +1,101 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.connector.catalog.SupportsDelete
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.DELETE
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle DELETE statements.
+ *
+ * If a table implements SupportsDelete and SupportsRowLevelOperations, we 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 @ DeleteFromIcebergTable(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 rewritePlan = operation match {
+            case _: SupportsDelta =>
+              throw new AnalysisException("Delta operations are currently not supported")
+            case _ =>
+              buildReplaceDataPlan(r, table, cond)
+          }
+          // keep the original relation in DELETE to try deleting using filters
+          DeleteFromIcebergTable(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,
+      table: RowLevelOperationTable,
+      cond: Option[Expression]): ReplaceData = {
+
+    // resolve all needed attrs (e.g. metadata attrs for grouping data on write)
+    val rowAttrs = relation.output
+    val metadataAttrs = resolveRequiredMetadataAttrs(relation, table.operation)
+
+    // construct a scan relation and include all required metadata columns
+    val scanAttrs = dedupAttrs(rowAttrs ++ metadataAttrs)
+    val scanRelation = relation.copy(table = table, output = scanAttrs)

Review comment:
       Ah, I thought it was the original table from my first reading. 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExprId
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.iceberg.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationInfoImpl
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import scala.collection.mutable
+
+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 dedupAttrs(attrs: Seq[AttributeReference]): Seq[AttributeReference] = {
+    val exprIds = mutable.Set.empty[ExprId]
+    attrs.flatMap { attr =>
+      if (exprIds.contains(attr.exprId)) {
+        None
+      } else {
+        exprIds += attr.exprId
+        Some(attr)
+      }
+    }
+  }
+
+  protected def resolveRequiredMetadataAttrs(
+      relation: DataSourceV2Relation,
+      operation: RowLevelOperation): Seq[AttributeReference] = {
+
+    ExtendedV2ExpressionUtils.resolveRefs[AttributeReference](
+      operation.requiredMetadataAttributes,
+      relation)
+  }
+
+  protected def resolveRowIdAttrs(

Review comment:
       Removed.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceData.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.connector.write.Write
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Replace data in an existing table.
+ */
+case class ReplaceData(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    write: Option[Write] = None) extends V2WriteCommandLike {
+
+  override lazy val resolved: Boolean = table.resolved && query.resolved && outputResolved
+  override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // the incoming query may include metadata columns
+  lazy val dataInput: Seq[Attribute] = {

Review comment:
       The projection is inserted only after the write is built as the extra columns can be part of the distribution and ordering requested by the write.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java
##########
@@ -189,16 +192,30 @@ public StreamingWrite toStreaming() {
 
   private Distribution buildRequiredDistribution() {
     if (overwriteFiles) {
-      throw new UnsupportedOperationException("Copy-on-write operations are temporarily not supported");
+      DistributionMode distributionMode = rowLevelDistributionMode();
+      return SparkDistributionAndOrderingUtil.buildCopyOnWriteDistribution(table, rowLevelCommand, distributionMode);
     } else {
       DistributionMode distributionMode = writeConf.distributionMode();
       return SparkDistributionAndOrderingUtil.buildRequiredDistribution(table, distributionMode);
     }
   }
 
+  private DistributionMode rowLevelDistributionMode() {
+    switch (rowLevelCommand) {
+      case DELETE:
+        return writeConf.copyOnWriteDeleteDistributionMode();
+      case UPDATE:
+        return writeConf.updateDistributionMode();
+      case MERGE:

Review comment:
       Removed whatever found.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -178,8 +178,25 @@ public DistributionMode distributionMode() {
     }
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.spec().isUnpartitioned() && table.sortOrder().isUnsorted()) {

Review comment:
       Yeah, that's true. But still, it's what the user is asking for... I'd probably go ahead and honor that request.
   
   Either way, I think that if we use range or hash we should add _file and _pos as the sort.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      // range distribution only makes sense if the sort order is set
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.sortOrder().isUnsorted()) {
+        return HASH;
+      } else {
+        return deleteMode;
+      }
+    } else {
+      // use hash distribution if write distribution is range or hash

Review comment:
       I agree with this one. No range unless delete mode is range.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java
##########
@@ -52,6 +56,7 @@ public static void startMetastoreAndSpark() {
         .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
         .config("spark.sql.shuffle.partitions", "4")
         .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true")
+        .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean()))

Review comment:
       We test with 3 catalogs and 3 different file formats so it seems reasonable.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.In
+import org.apache.spark.sql.catalyst.expressions.InSet
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.INSET
+import org.apache.spark.sql.catalyst.trees.TreePattern.NULL_LITERAL
+import org.apache.spark.sql.catalyst.trees.TreePattern.TRUE_OR_FALSE_LITERAL
+import org.apache.spark.sql.types.BooleanType
+import org.apache.spark.util.Utils
+
+/**
+ * A rule similar to ReplaceNullWithFalseInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] {

Review comment:
       Needed for metadata deletes.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandPruning.scala
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.dynamicpruning
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeMap
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.DynamicPruningSubquery
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.SubqueryExpression
+import org.apache.spark.sql.catalyst.planning.RewrittenRowLevelCommand
+import org.apache.spark.sql.catalyst.plans.LeftSemi
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.Sort
+import org.apache.spark.sql.catalyst.plans.logical.Subquery
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.PLAN_EXPRESSION
+import org.apache.spark.sql.catalyst.trees.TreePattern.SORT
+import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * A rule that adds a runtime filter for row-level commands.
+ *
+ * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account.
+ * Row-based rewrite plans are subject to usual runtime filtering.
+ */
+case class RowLevelCommandPruning(spark: SparkSession) 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 =>
+
+      // use reference equality to find exactly the required scan relations
+      val newRewritePlan = rewritePlan transformUp {
+        case r: DataSourceV2ScanRelation if r.scan eq scan =>
+          val pruningKeys = ExtendedV2ExpressionUtils.resolveRefs[Attribute](scan.filterAttributes, r)
+          val dynamicPruningCond = buildDynamicPruningCondition(r, command, pruningKeys)
+          val filter = Filter(dynamicPruningCond, r)
+          // always optimize dynamic filtering subqueries for row-level commands as it is important
+          // to rewrite introduced predicates as joins because Spark recently stopped optimizing
+          // dynamic subqueries to facilitate broadcast reuse
+          optimizeSubquery(filter)
+      }
+      command.withNewRewritePlan(newRewritePlan)
+  }
+
+  private def buildDynamicPruningCondition(
+      relation: DataSourceV2ScanRelation,
+      command: RowLevelCommand,
+      pruningKeys: Seq[Attribute]): Expression = {
+
+    // construct a filtering plan with the original scan relation
+    val cond = command.condition.getOrElse(Literal.TrueLiteral)
+    val matchingRowsPlan = command match {
+      case m: MergeIntoTable =>
+        Join(relation, m.sourceTable, LeftSemi, Some(cond), JoinHint.NONE)
+
+      case u: UpdateTable =>
+        // UPDATEs with subqueries may be rewritten using a UNION with two identical scan relations
+        // each scan relation will get its own dynamic filter that will be shared during execution
+        // the analyzer will assign different expr IDs for each scan relation output attributes
+        // that's why the condition may refer to invalid attr expr IDs and must be transformed
+        val attrMap = AttributeMap(u.table.output.zip(relation.output))
+        val transformedCond = cond transform {
+          case attr: AttributeReference if attrMap.contains(attr) => attrMap(attr)
+        }
+        Filter(transformedCond, relation)
+
+      case _ =>
+        Filter(cond, relation)

Review comment:
       It makes sense to me to use the original delete condition here for the dynamic pruning.
   
   I would prefer to make the command matching more explicit though, rather than doing this for anything that doesn't match `MergeIntoTable` or `UpdateTable`.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       I think that this works because the pre-CBO rules are executed once, so order matters. Adding the conversion to metadata delete before replacing the plan with the `ReplaceData` is important. Could you add a comment that notes 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -75,6 +76,22 @@ public void removeTables() {
     sql("DROP TABLE IF EXISTS deleted_dep");
   }
 
+  @Test
+  public void testDeleteWithFalseCondition() {
+    createAndInitUnpartitionedTable();
+
+    sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName);
+
+    sql("DELETE FROM %s WHERE id = 1 AND id > 20", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+    Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots()));

Review comment:
       Does this create a new snapshot?




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.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.optimizer
+
+import org.apache.spark.sql.catalyst.expressions.And
+import org.apache.spark.sql.catalyst.expressions.CaseWhen
+import org.apache.spark.sql.catalyst.expressions.Coalesce
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.If
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.Not
+import org.apache.spark.sql.catalyst.expressions.Or
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.TreePattern.CASE_WHEN
+import org.apache.spark.sql.catalyst.trees.TreePattern.IF
+import org.apache.spark.sql.types.BooleanType
+
+/**
+ * A rule similar to SimplifyConditionalsInPredicate in Spark but applies to Iceberg row-level commands.
+ */
+object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] {

Review comment:
       Is it possible to let the original rules simplify the parsed `DeleteFromTable` and only convert to `DeleteFromIcebergTable` after this has run? Something like this:
   
   ```scala
         parsedPlan match {
           case DeleteFromTable(UnresolvedIcebergTable(aliasedTable), Some(condition)) if isSimplified(cond) =>
             DeleteFromIcebergTable(aliasedTable, condition)
           case _ =>
             parsedPlan
         }
   
     ...
   
     private isSimplified(cond: Expression): Boolean = {
       !_.containsAnyPattern(CASE_WHEN, IF, NULL_LITERAL, TRUE_OR_FALSE_LITERAL, INSET)
     }
   ```




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java
##########
@@ -189,16 +192,30 @@ public StreamingWrite toStreaming() {
 
   private Distribution buildRequiredDistribution() {
     if (overwriteFiles) {
-      throw new UnsupportedOperationException("Copy-on-write operations are temporarily not supported");
+      DistributionMode distributionMode = rowLevelDistributionMode();
+      return SparkDistributionAndOrderingUtil.buildCopyOnWriteDistribution(table, rowLevelCommand, distributionMode);
     } else {
       DistributionMode distributionMode = writeConf.distributionMode();
       return SparkDistributionAndOrderingUtil.buildRequiredDistribution(table, distributionMode);
     }
   }
 
+  private DistributionMode rowLevelDistributionMode() {
+    switch (rowLevelCommand) {
+      case DELETE:
+        return writeConf.copyOnWriteDeleteDistributionMode();
+      case UPDATE:
+        return writeConf.updateDistributionMode();
+      case MERGE:

Review comment:
       Can we remove UPDATE and MERGE 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {

Review comment:
       I did not plan to have a custom method for copy-on-write but the question is whether we want to support `range` distribution mode for copy-on-write deletes. Since recently, there is a separate `DELETE_DISTRIBUTION_MODE` property and we need to define our behavior if it is set to `range`. I am debating whether that is useful. The only use case I can think of is if the cluster we issue deletes from is relatively small and we cannot hash all data per file into a single task. In that case, we may prefer to use `range` if the sort order is set.
   
   Any thoughts are welcome.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,79 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.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.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.types.StructType
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>

Review comment:
       Yeah, you got it correctly. That's why the scan planning is different.
   I'll add a comment.




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

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,79 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.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.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.types.StructType
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>

Review comment:
       Yeah, you got it correctly. That's why the scan planning is different.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)

Review comment:
       Correct. The idea is to report a row schema without metadata 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #3661:
URL: https://github.com/apache/iceberg/pull/3661#discussion_r762675350



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -35,9 +42,16 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // analyzer extensions
     extensions.injectResolutionRule { spark => ResolveProcedures(spark) }
     extensions.injectResolutionRule { _ => ProcedureArgumentCoercion }
+    extensions.injectResolutionRule { _ => RewriteDeleteFromTable }
 
     // optimizer extensions
+    extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate }
+    extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate }
+    extensions.injectPreCBORule { _ => OptimizeMetadataOnlyDeleteFromTable }

Review comment:
       +1 for adding a comment about why certain rules are added and their "preconditions" and associated "co-conditions" (even if it's Spark functionality we're relying on).
   
   It's very helpful to others who aren't as aware of Spark plan resolution and allows more people to effectively review and contribute in this area of the codebase.
   
   Particularly here where we're relying on order of operations for something that could possibly silently fail / misbehave if accidentally changed.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java
##########
@@ -189,16 +192,30 @@ public StreamingWrite toStreaming() {
 
   private Distribution buildRequiredDistribution() {
     if (overwriteFiles) {
-      throw new UnsupportedOperationException("Copy-on-write operations are temporarily not supported");
+      DistributionMode distributionMode = rowLevelDistributionMode();
+      return SparkDistributionAndOrderingUtil.buildCopyOnWriteDistribution(table, rowLevelCommand, distributionMode);
     } else {
       DistributionMode distributionMode = writeConf.distributionMode();
       return SparkDistributionAndOrderingUtil.buildRequiredDistribution(table, distributionMode);
     }
   }
 
+  private DistributionMode rowLevelDistributionMode() {
+    switch (rowLevelCommand) {
+      case DELETE:
+        return writeConf.copyOnWriteDeleteDistributionMode();
+      case UPDATE:
+        return writeConf.updateDistributionMode();
+      case MERGE:

Review comment:
       Will do a pass through the PR to remove such places.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(
+      table: Table,
+      plan: LogicalPlan,
+      allowScanDuplication: Boolean): Option[LogicalPlan] = {
+
+    val scanRelations = plan.collect {
+      case r: DataSourceV2Relation if r.table eq table => r
+      case r: DataSourceV2ScanRelation if r.relation.table eq table => r
+    }
+
+    // in some cases, the optimizer replaces the v2 scan relation with a local relation
+    // for example, there is no reason to query the table if the condition is always false
+    // that's why it is valid not to find the corresponding v2 scan relation
+
+    scanRelations match {
+      case relations if relations.isEmpty =>
+        None
+
+      case Seq(relation) =>
+        Some(relation)
+
+      case Seq(relation1: DataSourceV2Relation, relation2: DataSourceV2Relation)
+        if allowScanDuplication && (relation1.table eq relation2.table) =>
+        Some(relation1)
+
+      case Seq(relation1: DataSourceV2ScanRelation, relation2: DataSourceV2ScanRelation)
+        if allowScanDuplication && (relation1.scan eq relation2.scan) =>
+        Some(relation1)
+
+      case Seq(relation1, relation2) if allowScanDuplication =>
+        throw new AnalysisException(s"Row-level scan relations don't match: $relation1, $relation2")
+
+      case relations if allowScanDuplication =>
+        throw new AnalysisException(s"Expected 2 row-level scan relations: $relations")

Review comment:
       Removed 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala
##########
@@ -0,0 +1,79 @@
+/*
+ * 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
+import org.apache.spark.sql.catalyst.expressions.PredicateHelper
+import org.apache.spark.sql.catalyst.expressions.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.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.types.StructType
+
+object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with PredicateHelper {
+  import ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown {
+    case RewrittenRowLevelCommand(command, relation: DataSourceV2Relation, rewritePlan) =>

Review comment:
       If I understand correctly, this works for `DeleteFromTable` because the filter that is being pushed here is _not_ the filter from the rewrite plan (which is negated!) and is instead the filter from the command (`command.condition`).
   
   I think that's really important to call out in a comment. That was one of the first things I was trying to understand for this 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -163,8 +167,25 @@ public DistributionMode distributionMode() {
     return DistributionMode.fromName(modeName);
   }
 
-  public DistributionMode deleteDistributionMode() {
-    return rowLevelCommandDistributionMode(TableProperties.DELETE_DISTRIBUTION_MODE);
+  public DistributionMode copyOnWriteDeleteDistributionMode() {
+    String deleteModeName = confParser.stringConf()
+        .option(SparkWriteOptions.DISTRIBUTION_MODE)
+        .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE)
+        .parseOptional();
+
+    if (deleteModeName != null) {
+      // range distribution only makes sense if the sort order is set
+      DistributionMode deleteMode = DistributionMode.fromName(deleteModeName);
+      if (deleteMode == RANGE && table.sortOrder().isUnsorted()) {
+        return HASH;
+      } else {
+        return deleteMode;
+      }
+    } else {
+      // use hash distribution if write distribution is range or hash

Review comment:
       Why not inherit the range mode? If we support it explicitly anyway then it makes sense to use it 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(

Review comment:
       This is called `findScanRelation` but may return either a `DataSourceV2ScanRelation` or a relation that is not a `*ScanRelation`. Should this be `findRelationForTable` 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala
##########
@@ -90,6 +93,14 @@ object ExtendedV2Writes extends Rule[LogicalPlan] with PredicateHelper {
       }
       val newQuery = ExtendedDistributionAndOrderingUtils.prepareQuery(write, query, conf)
       o.copy(write = Some(write), query = newQuery)
+
+    case rd @ ReplaceData(r: DataSourceV2Relation, query, _, None) =>
+      val rowSchema = StructType.fromAttributes(rd.dataInput)
+      val writeBuilder = newWriteBuilder(r.table, rowSchema, Map.empty)
+      val write = writeBuilder.build()
+      // TODO: detect when query contains a shuffle and insert a round-robin repartitioning

Review comment:
       Okay, then we both agree to leave it out for now and reconsider later. I'll remove the TODO note.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.catalyst.plans.logical.UpdateTable
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      val allowScanDuplication = c match {
+        // group-based updates that rely on the union approach may have multiple identical scans
+        case _: UpdateTable if rewritePlan.isInstanceOf[ReplaceData] => true
+        case _ => false
+      }
+
+      rewritePlan match {
+        case replaceData: ReplaceData =>
+          replaceData.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>
+              val scanRelation = findScanRelation(table, replaceData.query, allowScanDuplication)
+              scanRelation.map((c, _, replaceData))
+            case _ =>
+              None
+          }
+      }
+
+    case _ =>
+      None
+  }
+
+  private def findScanRelation(

Review comment:
       I like "read", but this is minor.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);

Review comment:
       I matched the existing logic we had in 3.0. I guess we previously looked at the write distribution and if the table had no sort, was unpartitioned and the write distribution wasn't set, we assumed there is no reasonable distribution and ordering in the table so there was no reason trying to keep it.
   
   There are some benefits of using `hash` (assuming the user distributed and ordered the data manually) so I can change the default value.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      rewritePlan match {
+        case rd: ReplaceData =>
+          rd.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>

Review comment:
       It is weird, I agree. I've restructured this place to match what Spark does in other places.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.planning
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceData
+import org.apache.spark.sql.catalyst.plans.logical.RowLevelCommand
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+
+/**
+ * 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 DeleteFromIcebergTable);
+ *  - the scan relation in the rewrite plan that can be either DataSourceV2Relation or
+ *  DataSourceV2ScanRelation depending on whether the planning has already happened;
+ *  - the current rewrite plan.
+ */
+object RewrittenRowLevelCommand {
+  type ReturnType = (RowLevelCommand, LogicalPlan, LogicalPlan)
+
+  def unapply(plan: LogicalPlan): Option[ReturnType] = plan match {
+    case c: RowLevelCommand if c.rewritePlan.nonEmpty =>
+      val rewritePlan = c.rewritePlan.get
+
+      // both ReplaceData and WriteDelta reference a write relation
+      // but the corresponding scan relation should be at the bottom of the write plan
+      // both the write and scan relations will share the same RowLevelOperationTable object
+      // that's why it is safe to use reference equality to find the needed scan relation
+
+      rewritePlan match {
+        case rd: ReplaceData =>
+          rd.table match {
+            case DataSourceV2Relation(table, _, _, _, _) =>

Review comment:
       I agree. I've restructured this place to match what Spark does in other places.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);

Review comment:
       This is okay. I noted above that I'd probably lean toward respecting RANGE. But hash with _file and _pos seems reasonable.




-- 
This is an automated message from the 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #3661: Spark: Implement copy-on-write DELETE

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



##########
File path: spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
##########
@@ -296,6 +299,285 @@ public void testRangeWritePartitionedSortedTable() {
     checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
   }
 
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+    SortOrder[] expectedOrdering = new SortOrder[]{};
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() {
+    sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    table.replaceSortOrder()
+        .asc("id")
+        .asc("data")
+        .commit();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING)
+    };
+
+    Distribution expectedDistribution = Distributions.ordered(expectedOrdering);
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeletePartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " +
+        "USING iceberg " +
+        "PARTITIONED BY (date, days(ts))", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Distribution expectedDistribution = Distributions.unspecified();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testHashCopyOnWriteDeletePartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " +
+        "USING iceberg " +
+        "PARTITIONED BY (date, days(ts))", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH)
+        .commit();
+
+    Expression[] expectedClustering = new Expression[]{
+        Expressions.column(MetadataColumns.FILE_PATH.name()),
+    };
+    Distribution expectedDistribution = Distributions.clustered(expectedClustering);
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING)
+    };
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testRangeCopyOnWriteDeletePartitionedUnsortedTable() {
+    sql("CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " +
+        "USING iceberg " +
+        "PARTITIONED BY (date, days(ts))", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    table.updateProperties()
+        .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE)
+        .commit();
+
+    SortOrder[] expectedOrdering = new SortOrder[]{
+        Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING),
+        Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING)
+    };
+
+    Distribution expectedDistribution = Distributions.ordered(expectedOrdering);
+
+    checkCopyOnWriteDeleteDistributionAndOrdering(table, expectedDistribution, expectedOrdering);
+  }
+
+  @Test
+  public void testDefaultCopyOnWriteDeletePartitionedSortedTable() {

Review comment:
       This seems like the reasonable thing to do for nearly all cases where we might use no 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: issues-unsubscribe@iceberg.apache.org

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org