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 2020/12/16 22:36:20 UTC

[GitHub] [iceberg] dilipbiswal opened a new pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

dilipbiswal opened a new pull request #1947:
URL: https://github.com/apache/iceberg/pull/1947


   - Adds WIP support of MERGE INTO for spark leveraging the work done for DELETE by Anton.
   - This PR implements by doing copy-on-write.
   
   Plan:
   ```
   == Optimized Logical Plan ==
   ReplaceData RelationV2[key1#50, value1#51] file:///..., IcebergWrite(table=file:///..., format=PARQUET)
   +- MergeInto org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor@e1a150c, RelationV2[key1#50, value1#51] file:///...
      +- Join FullOuter, (key1#50 = key2#65)
         :- Project [key2#65, value2#66, true AS _source_row_present_#138]
         :  +- RelationV2[key2#65, value2#66] file:///...
         +- Project [key1#50, value1#51, true AS _target_row_present_#139]
            +- DynamicFileFilter
               :- RelationV2[key1#50, value1#51] file:///...
               +- Aggregate [_file_name_#137], [_file_name_#137]
                  +- Project [_file_name_#137]
                     +- Join Inner, (key1#50 = key2#65)
                        :- Filter isnotnull(key2#65)
                        :  +- RelationV2[key2#65] file:///...
                        +- Project [key1#50, input_file_name() AS _file_name_#137]
                           +- Filter isnotnull(key1#50)
                              +- RelationV2[key1#50] file:///...
   
   ``` 


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

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



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


[GitHub] [iceberg] dilipbiswal edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

Posted by GitBox <gi...@apache.org>.
dilipbiswal edited a comment on pull request #1947:
URL: https://github.com/apache/iceberg/pull/1947#issuecomment-753580792


   @aokolnychyi Regarding grouping the writes, I have made changes that groups the records based on partitioning and sort specification of the target table. I think it would be a reasonable initial implementation and we can iterate to improve ? Here are the changes. I have not tested it fully. 
   `https://github.com/dilipbiswal/iceberg/pull/new/merge_into_copyonwrite_group_sort` 
   Please see the last commit.
   @aokolnychyi @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.

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] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @aokolnychyi @rdblue 
   I have cleaned up the PR to implement the comments. This is the base PR that has core logic for MERGE INTO processing. I have opened two separate PRs 1) Cardinality check 2) Grouping and sorting rows before write. We can discuss any issues with the implementation of (1) or (2) separately in those PRs. Hope it is okay ?
   
   Here are the two PRs.
   https://github.com/apache/iceberg/pull/2021
   https://github.com/apache/iceberg/pull/2022
   
   Thanks
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"

Review comment:
       Shall we make these protected?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.scala
##########
@@ -0,0 +1,39 @@
+/*
+ * 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._
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+case class MergeInto(mergeIntoProcessor: MergeIntoParams,

Review comment:
       nit: I like Spark's way of formatting like this a bit more:
   
   ```
   case class MergeInto(
       mergeIntoProcessor: ...
       targetRelation: ...
       child: ...)
   ```




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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   Is there enough consensus on making the cardinality check optional to match Hive and to avoid an extra inner join for merge-on-read? I think it should be enabled by default to prevent correctness problems.
   
   I don't think we agreed on how to implement the cardinality check. I had some thoughts in [this](https://github.com/apache/iceberg/pull/1947#issuecomment-747450897) comment. @dilipbiswal @rdblue @RussellSpitzer, what is your take on this? How do you see it is implemented?
   
   @RussellSpitzer did mention a corner case where the accumulator approach consumes a lot of memory on the driver (if each executor has a substantially large set of unique files and they are brought to the driver and merged into a single set, which leads to basically having the same copies many times). I am not sure we can overcome it, though.
   
   


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

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoProcessor,

Review comment:
       I agree with that. I think we can address this in the end. This bit is working and I'd focus on other things 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.

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = actions.map(getClauseCondition),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notActions.map(getClauseCondition),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ Literal(false),
+          deleteOutput = targetOutputCols :+ Literal(true),
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {

Review comment:
       Shall we make the helper method private?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       It seems a bit odd to apply this projection because the target row will be deleted. It seems like we could use the same lazily-initialized row for every delete.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       We need to make sure there are analysis rules that guarantee the assumptions in this class. One possible issue that jumped out to both @aokolnychyi and I was that this assumes the expressions for insert and update actions are correct for the output of this node. We need to make sure that is the case.
   
   Originally, I asked on Slack how that validation was being done, but I saw Anton's comment about it and I thought that probably meant that it isn't being done. If there are already rules in Spark to resolve and validate the plan, then that's great but we need to identify them and make a note here that we're relying on those for correctness. I still suspect that there aren't rules in Spark doing this because this is running the analyzer on expressions.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoProcessor,

Review comment:
       Well, I overlooked that we use `MergeInto` node in `RewriteMergeInto`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))

Review comment:
       Also, we will need to extract the predicates from the match condition that only reference the target table. Because the scan relation is built here, we need to push all predicates that can be used to filter the target table in this method. It should be a matter of checking which expressions contain only references in the target table's attribute set.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)

Review comment:
       Let's clear this up in a follow-up.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,275 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  @Parameterized.Parameters(
+      name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, vectorized = {4}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        { "testhive", SparkCatalog.class.getName(),
+            ImmutableMap.of(
+                "type", "hive",
+                "default-namespace", "default"
+            ),
+            "parquet",
+            true
+        },
+        { "spark_catalog", SparkSessionCatalog.class.getName(),
+            ImmutableMap.of(
+                "type", "hive",
+                "default-namespace", "default",
+                "clients", "1",
+                "parquet-enabled", "false",
+                "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync
+            ),
+            "parquet",
+            false
+        }
+    };
+  }
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.MERGE_MODE, TableProperties.MERGE_MODE_DEFAULT);
+  }
+
+  @Before
+  public void createTables() {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO %s AS target " +
+                     "USING %s AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO %s AS target " +
+                     "USING %s AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-six"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO %s AS target " +
+            "USING %s AS source " +
+            "ON target.id = source.id " +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-six")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO %s AS target " +
+            "USING %s AS source " +
+            "ON target.id = source.id " +
+            "WHEN MATCHED AND target.id = 6 THEN DELETE";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-one")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testAllCauses() throws NoSuchTableException {
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO %s AS target " +
+                     "USING %s AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * " +
+                     "WHEN MATCHED AND target.id = 6 THEN DELETE " +
+                     "WHEN NOT MATCHED AND source.id = 2 THEN INSERT * ";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testAllCausesWithExplicitColumnSpecification() throws NoSuchTableException {
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO %s AS target " +
+            "USING %s AS source " +
+            "ON target.id = source.id " +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET target.id = source.id, target.dep = source.dep " +
+            "WHEN MATCHED AND target.id = 6 THEN DELETE " +
+            "WHEN NOT MATCHED AND source.id = 2 THEN INSERT (target.id, target.dep) VALUES (source.id, source.dep) ";
+
+    sql(sqlText, targetName, sourceName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testSourceCTE() throws NoSuchTableException {
+    Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop"));
+    Assume.assumeFalse(catalogName.equalsIgnoreCase("testhive"));
+
+    append(targetName, new Employee(2, "emp-id-two"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-3"), new Employee(1, "emp-id-2"), new Employee(5, "emp-id-6"));
+    String sourceCTE = "WITH cte1 AS (SELECT id + 1 AS id, dep FROM source)";
+    String sqlText = sourceCTE + " " + "MERGE INTO %s AS target " +

Review comment:
       Nit: it looks like there are unnecessary string literals. `" " + "MERGE ..."` can be updated to `" MERGE ..."`.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }

Review comment:
       @rdblue Sorry ryan, you refer to some comment below ? But i don't see any that relates to this particular comment. I understood the part about creating the pairs once. But i don't know how we can avoid passing the projections ?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       @rdblue Sorry Ryan. I didn't notice your comment on slack until now. So currently in my understanding Spark's Analyzer ensures that the `ResolveIntoTable` is fully resolved. 
   
   [code](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L1634-L1672)
   
   However, you are right that we don't do any semantics analysis on the plan currently. We should add it.
   
   




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)

Review comment:
       I guess `projectTargetCols` would still be needed for the case where the source row isn't present, but it would still make this a little simpler. Especially the `applyProjection` method.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +

Review comment:
       I'm fine either way, it was just a surprise.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       @rdblue I had thought about it. But couldn't think of a way to do it. How about, we create a materialized delete row once per partition like this :
   ```
   val deleteExpr = params.targetOutput.dropRight(1).map(e => Literal.default(e.dataType)) ++ Seq(Literal.create(true, BooleanType))
       val deletedRow1 = UnsafeProjection.create(deleteExpr)
       val deletedRow = deletedRow1.apply(null)
   ```
       




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.scala
##########
@@ -0,0 +1,39 @@
+/*
+ * 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._
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+
+case class MergeInto(mergeIntoProcessor: MergeIntoParams,

Review comment:
       nit: I like Spark's way of formatting like this:
   
   ```
   case class MergeInto(
       mergeIntoProcessor: ...
       targetRelation: ...
       child: ...)
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)

Review comment:
       The operation passed to the merge builder should be `merge`




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   cc @aokolnychyi @rdblue @mehtaashish23 


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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I can give it a try with and contribute rules that would align assignments + port our tests. It would be great if @dilipbiswal could work on the cardinality check and grouping of records on write.
   
   Once these are done, we can look into changing `MergeIntoProcessor`. 
   
   How does that sound?


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }

Review comment:
       I think it would be better to create these pairs just once instead of in every row. After this change and moving the projections into action cases (see comment below), this method signature would be much simpler:
   ```scala
     def applyProjection(
         actions: Seq[(BasePredicate, UnsafeProjection)],
         inputRow: InternalRow): InternalRow = {
       val pair = actions.find {
         case (predicate, _) => predicate.eval(inputRow)
       }
       ...
     }
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText);
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);

Review comment:
       Looks like there are other cases where SELECT statements run but are not used. Can you remove all of them?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.BasePredicate
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.UnaryExecNode
+
+case class MergeIntoExec(
+    mergeIntoParams: MergeIntoParams,
+    @transient targetRelation: DataSourceV2Relation,
+    override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoParams, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(
+     actions: Seq[(BasePredicate, UnsafeProjection)],
+     projectTargetCols: UnsafeProjection,
+     projectDeleteRow: UnsafeProjection,
+     inputRow: InternalRow,
+     targetRowNotPresent: Boolean): InternalRow = {
+
+    /**
+     * Find the first combination where the predicate evaluates to true.
+     * In case when there are overlapping condition in the MATCHED
+     * clauses, for the first one that satisfies the predicate, the
+     * corresponding action is applied. For example:
+     *
+     * WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
+     * WHEN MATCHED AND id = 5 OR id = 21 DELETE
+     *
+     * In above case, when id = 5, it applies both that matched predicates. In this
+     * case the first one we see is applied.
+     */

Review comment:
       Style: this is a doc comment because it starts with `/**`. Usually, multi-line comments in code would use `//  ` on each line.




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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


   Let me clone this change locally and go through it.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))

Review comment:
       Also, we will need to extract the predicates from the match condition that only reference the target table. Because the scan relation is built here, we need to push all predicates that can be used to filter the target table in this method. It should be a matter of checking which expressions contain only references in the target table's attribute set.
   
   We may want to do this in a follow-up. If so, it would be good to add a TODO comment for it.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * 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 java.util.UUID
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, JoinType}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation}
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+object RewriteMergeInto extends Rule[LogicalPlan]
+  with PredicateHelper
+  with Logging  {
+  val ROW_ID_COL = "_row_id_"
+  val FILE_NAME_COL = "_file_name_"
+  val SOURCE_ROW_PRESENT_COL = "_source_row_present_"
+  val TARGET_ROW_PRESENT_COL = "_target_row_present_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        // Find the files in target that matches the JOIN condition from source.
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, SOURCE_ROW_PRESENT_COL)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, TARGET_ROW_PRESENT_COL)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        val mergeIntoProcessor = new MergeIntoProcessor(
+          isSourceRowNotPresent = resolveExprs(Seq(col(SOURCE_ROW_PRESENT_COL).isNull.expr), joinPlan).head,
+          isTargetRowNotPresent = resolveExprs(Seq(col(TARGET_ROW_PRESENT_COL).isNull.expr), joinPlan).head,
+          matchedConditions = actions.map(resolveClauseCondition(_, joinPlan)),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols, joinPlan)),
+          notMatchedConditions = notActions.map(resolveClauseCondition(_, joinPlan)),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols, joinPlan)),
+          targetOutput = resolveExprs(targetOutputCols :+ Literal(false), joinPlan),
+          joinedAttributes = joinPlan.output
+        )
+
+        val mergePlan = MergeInto(mergeIntoProcessor, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  private def buildScanPlan(
+      table: Table,
+      output: Seq[AttributeReference],
+      mergeBuilder: MergeBuilder,
+      prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, output)
+
+    scan match {
+      case _: SupportsFileFilter =>
+        val matchingFilePlan = buildFileFilterPlan(prunedTargetPlan)
+        val dynamicFileFilter = DynamicFileFilter(scanRelation, matchingFilePlan)
+        dynamicFileFilter
+      case _ =>
+        scanRelation
+    }
+  }
+
+  private def newWriteInfo(schema: StructType): LogicalWriteInfo = {
+    val uuid = UUID.randomUUID()
+    LogicalWriteInfoImpl(queryId = uuid.toString, schema, CaseInsensitiveStringMap.empty)
+  }
+
+  private def buildFileFilterPlan(prunedTargetPlan: LogicalPlan): LogicalPlan = {
+    val fileAttr = findOutputAttr(prunedTargetPlan, FILE_NAME_COL)
+    Aggregate(Seq(fileAttr), Seq(fileAttr), prunedTargetPlan)
+  }
+
+  private def findOutputAttr(plan: LogicalPlan, attrName: String): Attribute = {
+    val resolver = SQLConf.get.resolver
+    plan.output.find(attr => resolver(attr.name, attrName)).getOrElse {
+      throw new AnalysisException(s"Cannot find $attrName in ${plan.output}")
+    }
+  }
+
+  private def resolveExprs(exprs: Seq[Expression], plan: LogicalPlan): Seq[Expression] = {
+    val spark = SparkSession.active
+    exprs.map { expr => resolveExpressionInternal(spark, expr, plan) }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {
+    target.schema.map { col =>
+      target.output.find(attr => SQLConf.get.resolver(attr.name, col.name)).getOrElse {
+        Alias(Literal(null, col.dataType), col.name)()
+      }
+    }
+  }
+
+  def actionOutput(clause: MergeAction,
+                   targetOutputCols: Seq[Expression],
+                   plan: LogicalPlan): Seq[Expression] = {
+    val exprs = clause match {
+      case u: UpdateAction =>
+        u.assignments.map(_.value) :+ Literal(false)
+      case _: DeleteAction =>
+        targetOutputCols :+ Literal(true)
+      case i: InsertAction =>
+        i.assignments.map(_.value) :+ Literal(false)

Review comment:
       As Anton noted, there needs to be validation that the assignments here (for both inserts and updates) match up with the `targetOutputCols`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)

Review comment:
       This line should use `merge`, instead of `delete` now. It should be 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.

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {

Review comment:
       This file is no longer used, so it can be 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.

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }

Review comment:
       This one: https://github.com/apache/iceberg/pull/1947#discussion_r558742391




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))

Review comment:
       We want this scan to be shared by both the file filter plan and the scan relation. I think that means that this can't accept the `matchedRowsPlan` / `prunedTargetPlan`:
   1. We don't want to plan the scan twice
   2. The scan built by `MergeBuilder` is going to have `_file` and `_pos` automatically added. If we want to reference those, we should use this scan.
   
   That means we'll have to add the inner join here, or pass the `scanRelation` to a function that produces the inner join. I think that's what @aokolnychyi was proposing in his version of this class.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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


   @rdblue @aokolnychyi 
   I have addressed most of the comments except one that i had trouble interpreting. I have marked some TODOs in the code for the ones that i will follow-up on. Here are the list:
   1. Pushdown local predicates from merge join condition into the dynamic file filter plan.
   2. Resolve _file column in an unambiguous fashion.
   3. Optimize delete projection to either 1. set null 2. empty row 3. constant row.
   4. Move tests to the test file created by Anton for merge.
   
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true

Review comment:
       I think that `find` does guarantee the order and gives you the first one, which would be the first case that matches. I didn't see the syntax with multiple clauses and extra predicates in the SQL 2003 spec, so this might just be an extension that some implementations use. We should definitely check the semantics.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @aokolnychyi Regarding grouping the writes, I have made changes that groups the records based on partitioning and sort specification of the table. I think it would be a reasonable initial implementation and we can iterate to improve ? Here are the changes 
   `https://github.com/dilipbiswal/iceberg/pull/new/merge_into_copyonwrite_group_sort` 
   Please see the last commit.
   @aokolnychyi @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.

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 edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I think this PR is a great start, @dilipbiswal!
   
   I noted the following points that we need to address for correctness (some may be done separately):
   - We should perform the cardinality check as SQL standard requires.
   - We should align the assignments according to the target columns. This also applies to UPDATE.
   - We should think about how to group data before writing.
   - We will need more tests.
   
   There are also good to have points (can be done in follow-ups if too much trouble):
   - We should use a LEFT ANTI join for merge statements with only `WHEN NOT MATCHED THEN INSERT` clause.
   - We should use a RIGHT OUTER join for merge statements with only `WHEN MATCHED` clauses.
   
   Let's discuss each point one by one.
   
   **Cardinality Check**
   
   SQL standard requires an exception to be thrown if the ON clause in MERGE is such that more than 1 row in source matches a row in target. See [this](https://issues.apache.org/jira/browse/HIVE-14949) Hive issue for more info.
   
   Some systems do the cardinality check all the time while some, like Hive, make it optional. I'd say we should make it optional and let users configure it in the table properties by adding `write.merge.cardinality-check.enabled` property (true by default). The main problem with the cardinality check is the performance penalty: it requires an inner join. We are already doing this inner join for copy-on-write to find matches so we can modify that code to also do the cardinality check at the same time. I don't think we need an inner join for merge-on-read, though. 
   
   To sum up, I'd vote for having a flag in table properties and make the cardinality check optional (just like Hive ACID).
   
   We need to think a bit about how we implement the cardinality check. Here, I am open to suggestions. One idea is to modify nodes for dynamic file filtering. For example, we can use `monotonically_increasing_id` until we have `row_id` metadata column, append it to rows in the target table before the inner join to find matches and then perform the cardinality check and collect matching files. In order to make this efficient, we should reuse as much work as possible.
   
   One way to do that is to leverage an accumulator to track matching files:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - define a udf that accepts the file name, adds it to the accumulator and retuns 1
   - group by `_row_id`, perform the cardinality check
   - access the accumulator to get the matching files
   
   Another way is like this:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - select `_row_id`,  `_file` into a separate plan
   - temporarily cache the created plan (or persist it on executor nodes)
   - perform 2 queries in parallel: one for cardinality check and one for the matching files
   - uncache/destroy the temp plan
   
   **Align Assignments**
   
   I don't think Spark aligns the assignments inside UPDATE or MERGE. We won't be able to support updating nested fields without it. We will probably need a separate rule for this. The same rule can be applied to UPDATE.
   
   **Group Data before Writing**
   
   We need to think about how to group data before writing new files with our updates and new records. One option is to group and order by partition columns. Another option is to group and order by the sort spec. The third option is to group updates and new records separately. Let's discuss it.


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }

Review comment:
       @rdblue Sorry ryan, you refer to some comment below ? But i don't see any comment below that relates to this particular comment. I understood the part about creating the pairs once. But i don't know how we can avoid passing the projections ?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoProcessor,

Review comment:
       I wonder whether we can use `MapPartitions` directly.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       @rdblue Can you please explain the idea bit more, specifically the `should be broken up into analysis rules to validate and update `MergeInto` plan` . Currently, we produce the `MergeInto` logical plan in the optimizer phase ? So we have gone past analysis at this point right ? The input SQL has already been parsed and resolved using `MergeIntoTable` by spark at this point i.e all the `mergeinto` inputs have been 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.

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       @rdblue One thing to note is that, the output of the outer join is target cols + source cols. So we have to project out the necessary target columns, i think.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,

Review comment:
       Okay, sounds fine to me. Let's still make the parameter changes, but remove the `partitioned` parameter.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.
+        val targetTableScan = buildScanPlan(target.table, target.output, mergeBuilder, None, matchingRowsPlanBuilder)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = matchedActions.map(getClauseCondition),
+          matchedOutputs = matchedActions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notMatchedActions.map(getClauseCondition),
+          notMatchedOutputs = notMatchedActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ FALSE_LITERAL,
+          deleteOutput = targetOutputCols :+ TRUE_LITERAL,
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  private def actionOutput(clause: MergeAction, targetOutputCols: Seq[Expression]): Seq[Expression] = {
+    clause match {
+      case u: UpdateAction =>
+        u.assignments.map(_.value) :+ FALSE_LITERAL
+      case _: DeleteAction =>
+        targetOutputCols :+ TRUE_LITERAL
+      case i: InsertAction =>
+        i.assignments.map(_.value) :+ FALSE_LITERAL
+    }
+  }
+
+  private def getClauseCondition(clause: MergeAction): Expression = {
+    clause.condition.getOrElse(Literal(true))

Review comment:
       This can use `TRUE_LITERAL`.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.

Review comment:
       @rdblue Yeah.. i saw it Ryan. I checked the spark code and there is an additional check for deterministic status of the expression. Not sure for delete statement, we need this check or not ? Wanted to think through and discuss with you and Anton and thats why put a to-do.




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

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,

Review comment:
       @rdblue Ryan, i had added the partitioning test cases in the grouby/sort pr ? I wasn't doing it via parameter as i was testing individual part and sort expressions. Can you please take a quick look ? I am thinking how those tests work with the proposed changes of creating tables in @Before and also with the introduction of partition parameter.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -43,6 +43,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // TODO: PullupCorrelatedPredicates should handle row-level operations
     extensions.injectOptimizerRule { _ => PullupCorrelatedPredicatesInRowLevelOperations }
     extensions.injectOptimizerRule { _ => RewriteDelete }
+    extensions.injectOptimizerRule { _ => RewriteMergeInto }

Review comment:
       @dilipbiswal, do we have to add MERGE operations to `PullupCorrelatedPredicatesInRowLevelOperations`? Could you test the current implementation with subqueries inside the merge as well as matched/not matched conditions?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())

Review comment:
       If you replace `lit(true).expr` with an expression, then this doesn't need to pull in `sql.functions._`. I added a constant:
   
   ```
     private val TRUE = Literal(true, BooleanType)
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))

Review comment:
       Nevermind on the second comment. Anton's update handles it.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = actions.map(getClauseCondition),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notActions.map(getClauseCondition),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ Literal(false),
+          deleteOutput = targetOutputCols :+ Literal(true),
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {

Review comment:
       Shall we make the helper methods private?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {

Review comment:
       @aokolnychyi If you are ok, i want to create a final pr to remove this test case and merge to `TestMerge` since the other two prs also add tests to this class. I want them to rebase okay without much trouble. Let me know please.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoProcessor,

Review comment:
       Do we need this node? It seems we rewrite the operation into `ReplaceData`, no?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records

Review comment:
       Does the comment apply? It looks like it is valid for DELETE but not really for MERGE.




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

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



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


[GitHub] [iceberg] dilipbiswal edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

Posted by GitBox <gi...@apache.org>.
dilipbiswal edited a comment on pull request #1947:
URL: https://github.com/apache/iceberg/pull/1947#issuecomment-753580792


   @aokolnychyi Regarding grouping the writes, I have made changes that groups the records based on partitioning and sort specification of the table. I think it would be a reasonable initial implementation and we can iterate to improve ? Here are the changes. I have not tested it fully. 
   `https://github.com/dilipbiswal/iceberg/pull/new/merge_into_copyonwrite_group_sort` 
   Please see the last commit.
   @aokolnychyi @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.

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] dilipbiswal commented on pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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


   @rdblue @aokolnychyi Thanks for the detailed review and all the help !!


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)

Review comment:
       @rdblue 
   
   1. ```projectTargetCols``` represents the expression that needs to be applied on the output of outer join which has columns from both the tables to only project the target output columns plus the deleted flag set to false.
   2. ```projectDeletedRow ``` does the same but with the "deleted flag". I think in the earlier comment we discussed possible ideas to optimize this (will address in follow-up)
   3. ``matchedPreds``` and ```notMatchedPred```  go hand in hand with their corresponding projections that is specified by the user in the `WHEN MATCHED ` and `WHEN NOT MATCHED` clauses.
   
   Given this background, can you please explain your idea a little bit ? 
   
    




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.

Review comment:
       @dilipbiswal, this extraction is already done in the `pushFilters` method that @aokolnychyi implemented for delete. That's one reason why this also passes down `target.output`. The filters that are pushed down are the ones that only reference those attributes:
   
   ```scala
       val tableAttrSet = AttributeSet(tableAttrs)
       val predicates = splitConjunctivePredicates(cond).filter(_.references.subsetOf(tableAttrSet))
       if (predicates.nonEmpty) {
         val normalizedPredicates = DataSourceStrategy.normalizeExprs(predicates, tableAttrs)
         PushDownUtils.pushFilters(scanBuilder, normalizedPredicates)
       }
   ```




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @rdblue Thanks for the comments. I will process them and get back with any questions. 


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {

Review comment:
       I think we better make methods inside this trait protected, not public.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala
##########
@@ -75,6 +76,8 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy {
     case ReplaceData(_, batchWrite, query) =>
       ReplaceDataExec(batchWrite, planLater(query)) :: Nil
 
+    case MergeInto(mergeIntoProcessor, targetRelation, child) =>
+      MergeIntoExec(mergeIntoProcessor, targetRelation, planLater(child)) :: Nil

Review comment:
       Nit: to make this class more readable, we've been separating cases with a blank line.




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>

Review comment:
       nit: I think `matchedActions` and `notMatchedActions` would be better names 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.

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > Please take a look at #1955. That exposes _pos so that we can use it.
   
   That is a great PR, let's get it in today.
   
   > If the table has a sort order, add a global sort of all the rows produced by the merge node.
   
   We have this option internally and it works well in some cases. There are a few things we need to be careful about, though.
   
   First, Spark will do a skew estimation step and the actual shuffle using two separate jobs. We don't want to recompute the merge join twice. We have a repartition stage after the join if a global sort on write is requested. While it does help a bit, it is not ideal. We have seen cases where the sort on write is by far the most expensive step of MERGE.
   
   Second, even when we do a global sort, the layout within partitions won't be ideal. So people will most likely have to compact again making the global sort during MERGE redundant.
   
   That's why we have to be careful about a global sort by default. I think this ultimately depends on the use case. Shall we make this configurable in table properties? How many query engines will follow it? Should that config be `copy-on-write` specific? I don't have answers to all the questions but it sounds reasonable to explore.
   
   At the same time, if we don't do the global sort, we may end up having too many small files after the operation. We can consider doing a repartition by the partition columns and sorting by the sort key but that will suffer if we have a lot of data for a single partition. It would be great to know the number of files and the size of data we need to rewrite per partition to make a good decision here.
   
   > If the table does not have a sort order, then add a default sort by _file, _pos, partition columns, and the MERGE condition's references.
   
   Sorting updated records by `_file` and `_pos` may be a bit tricky. For example, I have a file with columns (p, c1, c2, c3) in partition 'A' that is sorted by c1 and c2. If I have a merge command that updates c2 column (part of my sort key), my new records will be probably out of order if I sort by `_file` and `_pos`. That said, this is a fallback scenario so it may be not that big a deal. 
   
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");

Review comment:
       This should be the merge equivalent.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {

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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
##########
@@ -103,6 +103,7 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
   }
 
   private def buildFileFilterPlan(matchingRowsPlan: LogicalPlan): LogicalPlan = {
+    // TODO: For merge-into make sure _file is resolved only from target table.

Review comment:
       I think it should be fine. We should throw an exception if the table has a `_file` column, but that's something we can do later.




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

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.

Review comment:
       okay.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = actions.map(getClauseCondition),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notActions.map(getClauseCondition),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ Literal(false),
+          deleteOutput = targetOutputCols :+ Literal(true),
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {

Review comment:
       @aokolnychyi will remove. Thanks !!




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,

Review comment:
       This does not do predicate push down as we have for DELETE.
   Can we take the implementation from `RewriteDelete`?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText);
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText);
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);

Review comment:
       @rdblue sorry... had used this for debugging and forgot to remove it later.




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

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   BTW, I can work on some of these items in parallel so that we finish this ealier.


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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I agree with all of the points that @aokolnychyi brought up. I also have a few suggestions on how to do this more cleanly.
   
   > We should think about how to group data before writing.
   
   Please take a look at #1955. That exposes `_pos` so that we can use it. I would suggest the following:
   * If the table has a sort order, add a global sort of all the rows produced by the merge node.
   * If the table does not have a sort order, then add a default sort by `_file`, `_pos`, partition columns, and the `MERGE` condition's references. That way, existing rows that have non-null `_file` and `_pos` will mostly preserve the sort order in the source data files (except for changed columns). Inserted rows will have nulls for `_file` and `_pos` and will then be sorted by partition columns to minimize the number of output files, and then by columns from the merge condition because those are likely to be high cardinality (giving Spark the ability to split partitions).
   
   > We should align the assignments according to the target columns. This also applies to UPDATE.
   
   Agreed. I think that we should have a rule similar to the logic in [`TableOutputResolver`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TableOutputResolver.scala#L30). One branch would modify `UPDATE` and add expressions to pull unchanged column values from the existing row. Another branch would modify `INSERT` by rearranging the columns by name for `INSERT (_names_) VALUES (_vals_)`.
   
   And we should also have a `MergeOutputCheck` rule to assert that each `INSERT` or `UPDATE` action is aligned with the output of the merge node.
   
   Modifying and checking the logical plan in the analyzer like this will require analyzer rules and a logical plan that doesn't contain `MergeIntoProcessor`. We will need a logical plan that the normal analysis rules can run on. Then we can hopefully remove the resolution from the rewrite as well.
   
   > We will need more tests
   
   Definitely.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText);
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);

Review comment:
       This select has no effect, can you remove it?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 6 THEN DELETE";
+
+    sql(sqlText, "");

Review comment:
       @rdblue Sorry.. don't know why i was doing it. Will remove ?

##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 6 THEN DELETE";
+
+    sql(sqlText, "");

Review comment:
       @rdblue Sorry.. don't know why i was doing it. Will 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.

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   Okay, seems like we agree on doing a global sort (after an extra round-robin partitioning to make sure we don't execute the merge join twice) and having a table property under `write.merge.copy-on-write` to enable/disable the global sort in MERGE. If the global sort is disabled, we still need to perform a local sort. We will need to think about a proper name for the property but we can do this later. Seeing what performance penalty a global sort can introduce inside MERGE statements, I'd recommend not doing this by default but I can be convinced otherwise.
   
   > For sorting by _file and _pos, what if we only did that for existing rows? We can discard the columns for updated rows. That way we rewrite the data files as though the rows were deleted and append the inserts and updates together. We may even want to do this in all cases: always prepend _file and _pos to whatever sort order we inject.
   
   I think this is promising if we can easily nullify `_file` and `_pos` for updated rows and if Spark range estimation will do what we hope. Can someone estimate the complexity of implementing this? I'd support this idea.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)
+
+    def shouldDeleteRow(row: InternalRow): Boolean =
+      row.getBoolean(params.targetOutput.size - 1)
+
+
+    def processRow(inputRow: InternalRow): InternalRow = {

Review comment:
       I think comments here would help people who will maintain/contribute in the future.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)

Review comment:
       Is there a better name than `prunedTargetPlan`? What about `matchedRowsPlan`?




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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I know we spend a lot of time on grouping of data on write but it has been a pain point for a few use cases so I think it is important to get it correctly.
   
   cc @dilipbiswal @rdblue @RussellSpitzer 


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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I'll cover the missing rule while @dilipbiswal is working on the cardinality check and grouping on 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.

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true

Review comment:
       I think this will require further explanation. There will be up to 2 matched cases and we try to find the first one that matches? Does `find` guarantee the order of the traversal? 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.

Review comment:
       The only predicates that will be pushed are those that can be converted to Filter. I don't think any non-deterministic expressions can be converted so it should be fine.




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

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] dilipbiswal commented on a change in pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       @rdblue Sorry Ryan. I didn't notice your comment on slack until now. So currently Spark's Analyzer ensures that the `ResolveIntoTable` is fully resolved. 
   
   [code](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L1634-L1672)
   
   However, you are right that we don't do any semantics analysis on the plan currently. We should add it.
   
   




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

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



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


[GitHub] [iceberg] dilipbiswal edited a comment on pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

Posted by GitBox <gi...@apache.org>.
dilipbiswal edited a comment on pull request #1947:
URL: https://github.com/apache/iceberg/pull/1947#issuecomment-761754092


   @rdblue @aokolnychyi 
   I have addressed most of the comments except one that i had trouble interpreting. I have marked some TODOs in the code for the ones that i will follow-up on. Here are the list:
   1. Pushdown local predicates from merge join condition into the dynamic file filter plan. (I am testing this at the moment)
   2. Resolve _file column in an unambiguous fashion.
   3. Optimize delete projection to either 1. set null 2. empty row 3. constant row.
   4. Move tests to the test file created by Anton for merge.
   
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +

Review comment:
       Why did you choose to include the newlines?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = actions.map(getClauseCondition),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notActions.map(getClauseCondition),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ Literal(false),
+          deleteOutput = targetOutputCols :+ Literal(true),
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {
+    target.schema.map { col =>
+      target.output.find(attr => SQLConf.get.resolver(attr.name, col.name)).getOrElse {

Review comment:
       It is probably better to accept SQLConf in this rule like in `AlignMergeIntoTable`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.BasePredicate
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.UnaryExecNode
+
+case class MergeIntoExec(
+    mergeIntoParams: MergeIntoParams,
+    @transient targetRelation: DataSourceV2Relation,
+    override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoParams, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(
+     actions: Seq[(BasePredicate, UnsafeProjection)],
+     projectTargetCols: UnsafeProjection,
+     projectDeleteRow: UnsafeProjection,
+     inputRow: InternalRow,
+     targetRowNotPresent: Boolean): InternalRow = {
+
+
+    // Find the first combination where the predicate evaluates to true.
+    // In case when there are overlapping condition in the MATCHED
+    // clauses, for the first one that satisfies the predicate, the
+    // corresponding action is applied. For example:
+    //
+    // WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
+    // WHEN MATCHED AND id = 5 OR id = 21 DELETE
+    //
+    // In above case, when id = 5, it applies both that matched predicates. In this
+    // case the first one we see is applied.
+    //
+

Review comment:
       Nit: no need for an empty comment and an empty line.




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText);
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText);
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));

Review comment:
       The employee dep is identical for both records with id 6, so the assertion can't distinguish between the case where employee 6 is replaced or not. Could you update the original target data to `emp-id-six` and assert that it is unchanged because of the `target.id = 1` requirement?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @rdblue 
   > Since we are currently only implementing copy-on-write, I think it will be easier to do the cardinality check in the existing inner join.
   
   Oh.. since we have two options to choose from and were discussing which option to choose.. i thought doing a count check as a side thing (basically does the join twice) and raise an error as a start. But if we can pick one option between the two proposals now, i can give a try to implement it.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  private val ROW_FROM_SOURCE = "_row_from_source_"
+  private val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.

Review comment:
       This newline isn't needed. Lines up to 120 characters are allowed.

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  private val ROW_FROM_SOURCE = "_row_from_source_"
+  private val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)

Review comment:
       This plan must use the v2 scan relation. Otherwise, the `_file` column is not projected. This should be:
   
   ```scala
       val matchingRowsPlanBuilder = (rel: DataSourceV2ScanRelation) =>
           Join(source, rel, Inner, Some(cond), JoinHint.NONE)
   ```
   
   Then you can also remove `newProjectCols` and `newTargetTable`.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.BasePredicate
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.UnaryExecNode
+
+case class MergeIntoExec(
+    mergeIntoParams: MergeIntoParams,
+    @transient targetRelation: DataSourceV2Relation,
+    override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoParams, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(
+     actions: Seq[(BasePredicate, UnsafeProjection)],
+     projectTargetCols: UnsafeProjection,
+     projectDeleteRow: UnsafeProjection,
+     inputRow: InternalRow,
+     targetRowNotPresent: Boolean): InternalRow = {
+
+
+    // Find the first combination where the predicate evaluates to true.
+    // In case when there are overlapping condition in the MATCHED
+    // clauses, for the first one that satisfies the predicate, the
+    // corresponding action is applied. For example:
+    //
+    // WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
+    // WHEN MATCHED AND id = 5 OR id = 21 DELETE
+    //
+    // In above case, when id = 5, it applies both that matched predicates. In this
+    // case the first one we see is applied.
+    //
+
+    val pair = actions.find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(
+     params: MergeIntoParams,
+     rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)
+    val nonMatchedPairs =   notMatchedPreds zip notMatchedProjs
+    val matchedPairs = matchedPreds zip matchedProjs
+
+    def shouldDeleteRow(row: InternalRow): Boolean =
+      row.getBoolean(params.targetOutput.size - 1)
+

Review comment:
       Nit: extra newline.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))
+
+    scan match {
+      case filterable: SupportsFileFilter =>
+        val matchingFilePlan = buildFileFilterPlan(prunedTargetPlan)
+        val dynamicFileFilter = DynamicFileFilter(scanRelation, matchingFilePlan, filterable)
+        dynamicFileFilter
+      case _ =>
+        scanRelation
+    }
+  }
+
+  private def buildFileFilterPlan(prunedTargetPlan: LogicalPlan): LogicalPlan = {

Review comment:
       I think that this will require a reference to the target table because `_file` needs to come from the target table and not the source table. Right now it works because the target table is the only one with `_file` defined, but I think we should plan on selecting the right column if there are duplicates. If I were to define a column `_file` in the source data, it may not work.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._

Review comment:
       Could you use one import per line? That will help avoid git conflicts in the future.
   
   Also, we don't use wildcard imports in Iceberg because it can cause conflicts and it isn't clear where symbols are coming from when reading 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.

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
##########
@@ -103,6 +103,7 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
   }
 
   private def buildFileFilterPlan(matchingRowsPlan: LogicalPlan): LogicalPlan = {
+    // TODO: For merge-into make sure _file is resolved only from target table.

Review comment:
       @rdblue Don't we have an issue of the target table has a column named "_file" ? I was thinking we may need a way to solve it by creating a distinct co-relation name if _file is existing in the target relation's output ?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {

Review comment:
       @dilipbiswal, could you move these tests to `TestMerge` that was introduced recently?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
##########
@@ -54,12 +54,12 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
       table: Table,
       tableAttrs: Seq[AttributeReference],
       mergeBuilder: MergeBuilder,
-      cond: Expression,
+      cond: Option[Expression] = None,
       matchingRowsPlanBuilder: DataSourceV2ScanRelation => LogicalPlan): LogicalPlan = {
 
     val scanBuilder = mergeBuilder.asScanBuilder
 
-    pushFilters(scanBuilder, cond, tableAttrs)
+    cond.map(pushFilters(scanBuilder, _, tableAttrs))

Review comment:
       @rdblue ok.. i will change it for now. We will discuss the deterministic thing later.




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText);
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +
+                     "ON target.id = source.id " +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText);
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);

Review comment:
       `res` is not used in this test or others. Can you remove this line?




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

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 edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I think this PR is a great start, @dilipbiswal!
   
   I noted the following points that we need to address for correctness (some may be done separately):
   - We should perform the cardinality check as SQL standard requires.
   - We should align the assignments according to the target columns. This also applies to UPDATE.
   - We should think about how to group data before writing.
   - We will need more tests.
   
   There are also good to have points (can be done in follow-ups if too much trouble):
   - We should use a LEFT ANTI join for merge statements with only `WHEN NOT MATCHED THEN INSERT` clause.
   - We should use a RIGHT OUTER join for merge statements with only `WHEN MATCHED` clauses.
   
   Let's discuss each point one by one.
   
   **Cardinality check**
   
   SQL standard requires an exception to be thrown if the ON clause in MERGE is such that more than 1 row in source matches a row in target. See [this](https://issues.apache.org/jira/browse/HIVE-14949) Hive issue for more info.
   
   Some systems do the cardinality check all the time while some, like Hive, make it optional. I'd say we should make it optional and let users configure it in the table properties by adding `write.merge.cardinality-check.enabled` property (true by default). The main problem with the cardinality check is the performance penalty: it requires an inner join. We are already doing this inner join for copy-on-write to find matches so we can modify that code to also do the cardinality check at the same time. I don't think we need an inner join for merge-on-read, though. 
   
   To sum up, I'd vote for having a flag in table properties and make the cardinality check optional (just like Hive ACID).
   
   We need to think a bit about how we implement the cardinality check. Here, I am open to suggestions. One idea is to modify nodes for dynamic file filtering. For example, we can use `monotonically_increasing_id` until we have `row_id` metadata column, append it to rows in the target table before the inner join to find matches and then perform the cardinality check and collect matching files. In order to make this efficient, we should reuse as much work as possible.
   
   One way to do that is to leverage an accumulator to track matching files:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - define a udf that accepts the file name, adds it to the accumulator and retuns 1
   - group by `_row_id`, perform the cardinality check
   - access the accumulator to get the matching files
   
   Another way is like this:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - select `_row_id`,  `_file` into a separate plan
   - temporarily cache the created plan (or persist it on executor nodes)
   - perform 2 queries in parallel: one for cardinality check and one for the matching files
   - uncache/destroy the temp plan
   
   **Align assignments**
   
   I don't think Spark aligns the assignments inside UPDATE or MERGE. We won't be able to support updating nested fields without it. We will probably need a separate rule for this. The same rule can be applied to UPDATE.
   
   **Group data before writing**
   
   We need to think about how to group data before writing new files with our updates and new records. One option is to group and order by partition columns. Another option is to group and order by the sort spec. The third option is to group updates and new records separately. Let's discuss it.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * 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 java.util.UUID
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, JoinType}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation}
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+object RewriteMergeInto extends Rule[LogicalPlan]
+  with PredicateHelper
+  with Logging  {
+  val ROW_ID_COL = "_row_id_"

Review comment:
       nit: these vals can be private




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 6 THEN DELETE";
+
+    sql(sqlText, "");

Review comment:
       Why is this passing an empty string?




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

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



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


[GitHub] [iceberg] rdblue merged pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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


   


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,

Review comment:
       ok.. will do. thank you Ryan for doing such a thorough review. Really appreciate !!




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,

Review comment:
       I don't think that there is a need to test this with both Hive and Hadoop catalogs or with all 3 formats, since the main thing that needs to be tested is conversion and Spark behavior. Also, this doesn't test partitioned tables at all. To fix those, I think this should customize parameters:
   
   ```java
     @Parameterized.Parameters(
         name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, vectorized = {4}, partitioned = {5}")
     public static Object[][] parameters() {
       return new Object[][] {
           { "testhive", SparkCatalog.class.getName(),
               ImmutableMap.of(
                   "type", "hive",
                   "default-namespace", "default"
               ),
               "parquet",
               true,
               false
           },
           { "spark_catalog", SparkSessionCatalog.class.getName(),
               ImmutableMap.of(
                   "type", "hive",
                   "default-namespace", "default",
                   "clients", "1",
                   "parquet-enabled", "false",
                   "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync
               ),
               "parquet",
               false,
               true
           }
       };
     }
   
     public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
                               String fileFormat, Boolean vectorized, Boolean partitioned) {
       super(catalogName, implementation, config, fileFormat, vectorized);
       this.partitioned = partitioned;
       this.sourceName = tableName("source");
       this.targetName = tableName("target");
     }
   ```
   
   I also added a `partitioned` boolean and moved all of the `createAndInit` calls into a `@Before`:
   
   ```java
     @Before
     public void createTables() {
       if (partitioned) {
         createAndInitPartitionedTargetTable(targetName);
       } else {
         createAndInitUnPartitionedTargetTable(targetName);
       }
       createAndInitSourceTable(sourceName);
     }
   ```
   
   With those changes, tests run faster and cover partitioned 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.

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target " +
+                     "USING " + sourceName + " AS source " +

Review comment:
       Rather than embedding the names directly, you can pass them to the `sql` method, like `removeTables` does:
   
   ```java
   sql("MERGE INTO %s AS target USING %s AS source ...", targetName, sourceName);
   ```




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)

Review comment:
       These last two projections are only needed when `notMatchedPreds` or `matchedPreds` does not have a default case, i.e. `lit(true)`.
   
   In the rewrite, there is also a function, `getClauseCondition`, that fills in `lit(true)` if there is no clause condition. But I don't think that any predicates _after_ the `true` condition are dropped.
   
   I think we could simplify the logic here and avoid extra clauses by ensuring that both `matchedPreds` and `notMatchedPreds` end with `lit(true)`. Then this class would not need to account for the case where no predicate matches and we wouldn't have extra predicates passed through.




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

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



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


[GitHub] [iceberg] aokolnychyi edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I know we spend a lot of time on grouping of data on write but it has been a pain point for a number of use cases so I think it is important to get it correctly.
   
   cc @dilipbiswal @rdblue @RussellSpitzer 


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeIntoTable.java
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.iceberg.spark.extensions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
+import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED;
+
+public class TestMergeIntoTable extends SparkRowLevelOperationsTestBase {
+  private final String sourceName;
+  private final String targetName;
+
+  public TestMergeIntoTable(String catalogName, String implementation, Map<String, String> config,
+                            String fileFormat, Boolean vectorized) {
+    super(catalogName, implementation, config, fileFormat, vectorized);
+    this.sourceName = tableName("source");
+    this.targetName = tableName("target");
+  }
+
+  @BeforeClass
+  public static void setupSparkConf() {
+    spark.conf().set("spark.sql.shuffle.partitions", "4");
+  }
+
+  protected Map<String, String> extraTableProperties() {
+    return ImmutableMap.of(TableProperties.DELETE_MODE, "copy-on-write");
+  }
+
+  @After
+  public void removeTables() {
+    sql("DROP TABLE IF EXISTS %s", targetName);
+    sql("DROP TABLE IF EXISTS %s", sourceName);
+  }
+
+  @Test
+  public void testEmptyTargetInsertAllNonMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED THEN INSERT * ";
+
+    sql(sqlText, "");
+    sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testEmptyTargetInsertOnlyMatchingRows() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(sourceName, new Employee(1, "emp-id-1"), new Employee(2, "emp-id-2"), new Employee(3, "emp-id-3"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+                     "USING " + sourceName + " AS source \n" +
+                     "ON target.id = source.id \n" +
+                     "WHEN NOT MATCHED AND (source.id >= 2) THEN INSERT * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(2, "emp-id-2"), row(3, "emp-id-3")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyUpdate() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +
+            "USING " + sourceName + " AS source \n" +
+            "ON target.id = source.id \n" +
+            "WHEN MATCHED AND target.id = 1 THEN UPDATE SET * ";
+
+    sql(sqlText, "");
+    List<Object[]> res = sql("SELECT * FROM %s ORDER BY id, dep", targetName);
+    assertEquals("Should have expected rows",
+            ImmutableList.of(row(1, "emp-id-1"), row(6, "emp-id-6")),
+            sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", targetName));
+  }
+
+  @Test
+  public void testOnlyDelete() throws NoSuchTableException {
+    createAndInitUnPartitionedTargetTable(targetName);
+    createAndInitSourceTable(sourceName);
+    append(targetName, new Employee(1, "emp-id-one"), new Employee(6, "emp-id-6"));
+    append(sourceName, new Employee(2, "emp-id-2"), new Employee(1, "emp-id-1"), new Employee(6, "emp-id-6"));
+    String sqlText = "MERGE INTO " + targetName + " AS target \n" +

Review comment:
       @rdblue i guess, its not required. I thought if we print out the SQL as part of some error or debugging, it will just format better. I will remove the newlines.




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @aokolnychyi, I agree with the idea to have a flag to disable global sort. Probably best to do this specific to copy-on-write because delta writes will need to be sorted by `_file` and `_pos` for deletes and we expect the inserts to be much, much smaller than the copy-on-write data. If we aren't rewriting retained rows, I think the global sort (with a repartition as you said) would be much cheaper.
   
   For sorting by `_file` and `_pos`, what if we only did that for existing rows? We can discard the columns for updated rows. That way we rewrite the data files as though the rows were deleted and append the inserts and updates together. We may even want to do this in all cases: always prepend `_file` and `_pos` to whatever sort order we inject.


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

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



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


[GitHub] [iceberg] RussellSpitzer commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   One thing we've been talking about it a bit is whether or not it would be useful to tune the write portion of this. For example it may be helpful to have an independent parameter for tuning the shuffle parameters when grouping the results before writing. This probably would be good to tune with a non-spark parameter so that users can customize it apart from the normal spark.sql.shuffle parameter.


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true

Review comment:
       @aokolnychyi @rdblue 
   In my understanding, if we have overlapping conditions in two match branches i.e for example ..
   
   WHEN MATCHED AND id > 0 and id < 10  UPDATE *
   WHEN MATCHED AND id = 5  or id = 21 DELETE
   
   Then we will just honor the first match and so if a row comes with id = 5, then we will update. And if a row with id 21 is present, then we will delete.
   
   
   




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.BasePredicate
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.UnaryExecNode
+
+case class MergeIntoExec(
+    mergeIntoParams: MergeIntoParams,
+    @transient targetRelation: DataSourceV2Relation,
+    override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoParams, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(
+     actions: Seq[(BasePredicate, UnsafeProjection)],
+     projectTargetCols: UnsafeProjection,
+     projectDeleteRow: UnsafeProjection,
+     inputRow: InternalRow,
+     targetRowNotPresent: Boolean): InternalRow = {
+
+    /**
+     * Find the first combination where the predicate evaluates to true.
+     * In case when there are overlapping condition in the MATCHED
+     * clauses, for the first one that satisfies the predicate, the
+     * corresponding action is applied. For example:
+     *
+     * WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
+     * WHEN MATCHED AND id = 5 OR id = 21 DELETE
+     *
+     * In above case, when id = 5, it applies both that matched predicates. In this
+     * case the first one we see is applied.
+     */
+
+    val pair = actions.find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(
+     params: MergeIntoParams,
+     rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)
+    val nonMatchedPairs =   notMatchedPreds zip notMatchedProjs
+    val matchedPairs = matchedPreds zip matchedProjs
+
+    def shouldDeleteRow(row: InternalRow): Boolean =
+      row.getBoolean(params.targetOutput.size - 1)
+
+
+    /**
+     * This method is responsible for processing a input row to emit the resultant row with an
+     * additional column that indicates whether the row is going to be included in the final
+     * output of merge or not.
+     * 1. If there is a target row for which there is no corresponding source row (join condition not met)
+     *    - Only project the target columns with deleted flag set to false.
+     * 2. If there is a source row for which there is no corresponding target row (join condition not met)
+     *    - Apply the not matched actions (i.e INSERT actions) if non match conditions are met.
+     * 3. If there is a source row for which there is a corresponding target row (join condition met)
+     *    - Apply the matched actions (i.e DELETE or UPDATE actions) if match conditions are met.
+     */
+    def processRow(inputRow: InternalRow): InternalRow = {
+      if (isSourceRowNotPresentPred.eval(inputRow)) {
+        projectTargetCols.apply(inputRow)
+      } else if (isTargetRowNotPresentPred.eval(inputRow)) {
+        applyProjection(nonMatchedPairs, projectTargetCols, projectDeletedRow, inputRow, true)
+      } else {
+        applyProjection(matchedPairs, projectTargetCols, projectDeletedRow, inputRow, false)
+      }
+    }
+
+    rowIterator
+      .map(processRow)
+      .filter(!shouldDeleteRow(_))

Review comment:
       Nit: could use `filterNot` instead.

##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.BasePredicate
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.UnaryExecNode
+
+case class MergeIntoExec(
+    mergeIntoParams: MergeIntoParams,
+    @transient targetRelation: DataSourceV2Relation,
+    override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoParams, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(
+     actions: Seq[(BasePredicate, UnsafeProjection)],
+     projectTargetCols: UnsafeProjection,
+     projectDeleteRow: UnsafeProjection,
+     inputRow: InternalRow,
+     targetRowNotPresent: Boolean): InternalRow = {
+
+    /**
+     * Find the first combination where the predicate evaluates to true.
+     * In case when there are overlapping condition in the MATCHED
+     * clauses, for the first one that satisfies the predicate, the
+     * corresponding action is applied. For example:
+     *
+     * WHEN MATCHED AND id > 1 AND id < 10 UPDATE *
+     * WHEN MATCHED AND id = 5 OR id = 21 DELETE
+     *
+     * In above case, when id = 5, it applies both that matched predicates. In this
+     * case the first one we see is applied.
+     */
+
+    val pair = actions.find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(
+     params: MergeIntoParams,
+     rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)
+    val nonMatchedPairs =   notMatchedPreds zip notMatchedProjs
+    val matchedPairs = matchedPreds zip matchedProjs
+
+    def shouldDeleteRow(row: InternalRow): Boolean =
+      row.getBoolean(params.targetOutput.size - 1)
+
+
+    /**
+     * This method is responsible for processing a input row to emit the resultant row with an
+     * additional column that indicates whether the row is going to be included in the final
+     * output of merge or not.
+     * 1. If there is a target row for which there is no corresponding source row (join condition not met)
+     *    - Only project the target columns with deleted flag set to false.
+     * 2. If there is a source row for which there is no corresponding target row (join condition not met)
+     *    - Apply the not matched actions (i.e INSERT actions) if non match conditions are met.
+     * 3. If there is a source row for which there is a corresponding target row (join condition met)
+     *    - Apply the matched actions (i.e DELETE or UPDATE actions) if match conditions are met.
+     */
+    def processRow(inputRow: InternalRow): InternalRow = {
+      if (isSourceRowNotPresentPred.eval(inputRow)) {
+        projectTargetCols.apply(inputRow)
+      } else if (isTargetRowNotPresentPred.eval(inputRow)) {
+        applyProjection(nonMatchedPairs, projectTargetCols, projectDeletedRow, inputRow, true)
+      } else {
+        applyProjection(matchedPairs, projectTargetCols, projectDeletedRow, inputRow, false)
+      }
+    }
+
+    rowIterator
+      .map(processRow)
+      .filter(!shouldDeleteRow(_))

Review comment:
       Nit: could use `filterNot(shouldDeleteRow)` 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.

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] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @aokolnychyi @RussellSpitzer 
   Thanks for those excellent comments. I am going to process them and get back  :-)


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       What if we just returned `null` instead?
   
   I think the problem is that this is trying to create one output for each input row, then filtering happens afterward. An extra column is added to signal that the row should be kept or not. But we don't need to copy the row if it is going to be removed. We also don't need to copy incoming target rows just to add a `true` at the end if they are going to be kept.
   
   So what if we changed all of the delete cases to produce `null` instead?
   
   Let's not worry about this for now, but I'll open a PR after this is merged to simplify and avoid some of the copies.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.analysis.Resolver
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.InputFileName
+import org.apache.spark.sql.catalyst.expressions.IsNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+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.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeInto
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+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.UpdateAction
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.BooleanType
+
+case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+  private val TRUE_LITERAL = Literal(true, BooleanType)
+  private val FALSE_LITERAL = Literal(false, BooleanType)
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def resolver: Resolver = conf.resolver
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo)
+        val matchingRowsPlanBuilder = (_: DataSourceV2ScanRelation) =>
+          Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        // TODO - extract the local predicates that references the target from the join condition and
+        // pass to buildScanPlan to ensure push-down.

Review comment:
       @rdblue Yeah.. i saw it Ryan. I checked the spark code and there is an additional check for deterministic status of the expression. Not sure for delete statement, we need this check or not ? Wanted to think through and discuss and thats why put a to-do.




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

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,164 @@
+/*
+ * 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 java.util.UUID
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.{AnalysisException, SparkSession}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, JoinType}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation}
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types._
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+object RewriteMergeInto extends Rule[LogicalPlan]
+  with PredicateHelper
+  with Logging  {
+  val ROW_ID_COL = "_row_id_"
+  val FILE_NAME_COL = "_file_name_"
+  val SOURCE_ROW_PRESENT_COL = "_source_row_present_"
+  val TARGET_ROW_PRESENT_COL = "_target_row_present_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        // Find the files in target that matches the JOIN condition from source.
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)

Review comment:
       It would be helpful to group some of these plan nodes into sections, like in `RewriteDelete` where methods like `buildFileFilterPlan` and `buildScanPlan` give good context for what plans are being constructed and how they will be used.




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

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



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


[GitHub] [iceberg] aokolnychyi edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > Please take a look at #1955. That exposes _pos so that we can use it.
   
   That is a great PR, let's get it in today.
   
   > If the table has a sort order, add a global sort of all the rows produced by the merge node.
   
   We have this option internally and it works well in some cases. There are a few things we need to be careful about, though.
   
   First, Spark will do a skew estimation step and the actual shuffle using two separate jobs. We don't want to recompute the merge join twice. Internally, we add a repartition stage after the join if a global sort on write is requested. While it does help a bit, it is not ideal. We have seen cases where the sort on write is by far the most expensive step of MERGE.
   
   Second, even when we do a global sort, the layout within partitions won't be ideal. So people will most likely have to compact again making the global sort during MERGE redundant.
   
   That's why we have to be careful about a global sort by default. I think this ultimately depends on the use case. Shall we make this configurable in table properties? How many query engines will follow it? Should that config be `copy-on-write` specific? I don't have answers to all the questions but it sounds reasonable to explore.
   
   At the same time, if we don't do the global sort, we may end up having too many small files after the operation. We can consider doing a repartition by the partition columns and sorting by the sort key but that will suffer if we have a lot of data for a single partition. It would be great to know the number of files and the size of data we need to rewrite per partition to make a good decision here.
   
   > If the table does not have a sort order, then add a default sort by _file, _pos, partition columns, and the MERGE condition's references.
   
   Sorting updated records by `_file` and `_pos` may be a bit tricky. For example, I have a file with columns (p, c1, c2, c3) in partition 'A' that is sorted by c1 and c2. If I have a merge command that updates c2 column (part of my sort key), my new records will be probably out of order if I sort by `_file` and `_pos`. That said, this is a fallback scenario so it may be not that big a deal. 
   
   


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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   Ack.


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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I’ve been thinking about grouping of data on write during copy-on-write operations (merge-on-read is a different story).
   
   Right now, we only have a sort order in the table metadata. However, we will probably add a way to represent distribution since Spark will have such a concept. I think global and local sorts don’t address all use cases. We will want to request hash distribution on write in some cases (it is cheaper than the global sort and works well if the data size per partition is small and does not have to be split into multiple tasks). This applies to inserts as well as to other operations like updates.
   
   Since there will be a concept of distribution controlled by the user, the idea of leveraging both the distribution and sort order during row-level operations seems promising to me.
   
   **DELETE**
   
   Delete is an operation that does not change the order of data so we should be fine with just file and pos metadata columns.
   
   In master, we do a global sort by file and pos that is the most expensive option. I think we can switch to hash-partitioning by file and local sort by file and pos. Yes, a global sort would co-locate files from same partitions next to each other but I don’t think it is worth the price of the range-based shuffle. I’d be in favor of faster deletes and doing a compaction later instead of doing a global sort during deletes. The global sort won’t eliminate the need for compacting and will make deletes more expensive which would increase the chances of concurrent conflicts.
   
   In addition, I’d offer a table property specific to copy-on-write deletes to disable the shuffle step. If people want to have even faster deletes by skipping the shuffle, we should let them do that. They will have to compact more aggressively.
   
   **UPDATE**
   
   Update is the first operation that potentially changes the order of data. That’s why we should take the distribution and order into account. Our intention here is to group/sort rows that did not change by file and pos to preserve their original ordering and apply the distribution and order to updated records. If the user asks for hash-based distribution during inserts, most likely he/she wants to apply it during updates too.
   
   I’d consider the following options:
   - If the user asks for a global sort during inserts, do a range-based shuffle by `file`, `pos`, `if (file is null) sort_col_1 else null`, `if (file is null) sort_col_2 else null` and a local sort by the same attributes.
   - If the user asks for hash partitioning and local sort during inserts, do a hash-based shuffle by `file`, `if (file is null) dist_col_1 else null`, `if (file is null) dist_col_2 else null`, etc and a local sort by `file`, `pos`, `if (file is null) sort_col_1 else null`, `if (file is null) sort_col_2 else null` where `file` and `pos` columns would be `null` for updated records.
   - If the user asks for a local sort during inserts, do a local sort.
   - Add a table property specific to copy-on-write updates to ignore the configured distribution.
   
   **MERGE**
   
   Merge is similar to update. We should consider new and updated records together.


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       @rdblue OK. 




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {

Review comment:
       When I was working on UPDATE, I also created a parent trait for row-level ops.
   
   What about a more specific name, like `RewriteRowLevelOperation` or similar?




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)
+        val targetTableScan =  buildScanPlan(target.table, target.output, mergeBuilder, prunedTargetPlan)
+
+        // Construct an outer join to help track changes in source and target.
+        // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable.
+        val sourceTableProj = source.output ++ Seq(Alias(lit(true).expr, ROW_FROM_SOURCE)())
+        val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, ROW_FROM_TARGET)())
+        val newTargetTableScan = Project(targetTableProj, targetTableScan)
+        val newSourceTableScan = Project(sourceTableProj, source)
+        val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE)
+
+        // Construct the plan to replace the data based on the output of `MergeInto`
+        val mergeParams = MergeIntoParams(
+          isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_SOURCE)),
+          isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)),
+          matchedConditions = actions.map(getClauseCondition),
+          matchedOutputs = actions.map(actionOutput(_, targetOutputCols)),
+          notMatchedConditions = notActions.map(getClauseCondition),
+          notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)),
+          targetOutput = targetOutputCols :+ Literal(false),
+          deleteOutput = targetOutputCols :+ Literal(true),
+          joinedAttributes = joinPlan.output
+        )
+        val mergePlan = MergeInto(mergeParams, target, joinPlan)
+        val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch()
+        ReplaceData(target, batchWrite, mergePlan)
+    }
+  }
+
+  def getTargetOutputCols(target: DataSourceV2Relation): Seq[NamedExpression] = {

Review comment:
       Is this method actually used?




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I looked into resolution and there is a rule in Spark: https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L1682-L1710
   
   Looks like if the assignments are out of order or a subset of the output columns, the expressions are left as-is. If there are no assignments, then the source table's columns are used to set the output columns by position, using an `Attribute` from the target table as the LHS.
   
   We will need an analyzer rule that fills in the missing assignments for update, checks the order of assignments by name, and validates that inserts are complete. I also think that this rule should convert to a different MergeInto logical plan. The plan in Spark is not sufficient because it considers the plan resolved when assignments are resolved, not when the assignments actually produce the expected output. That's strange because resolution produces assignments when there aren't any, but allows them to be missing when some are present.


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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > I can give it a try with and contribute rules that would align assignments + port our tests. It would be great if @dilipbiswal could work on the cardinality check and grouping of records on write.
   
   Sounds good to me Anton.


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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > About count implementation Anton, i was thinking to implement it without the optimization as a first cut and optimize it in a follow-up.
   
   If I understand correctly, it is actually easier to do the optimization now because the optimization only requires changes in merge-on-read. Anton said this:
   
   > We are already doing this inner join for copy-on-write to find matches so we can modify that code to also do the cardinality check at the same time.
   
   Since we are currently only implementing copy-on-write, I think it will be easier to do the cardinality check in the existing inner join.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDelete.scala
##########
@@ -56,7 +56,7 @@ case class RewriteDelete(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRo
       d
 
     // rewrite all operations that require reading the table to delete records
-    case DeleteFromTable(r: DataSourceV2Relation, Some(cond)) =>
+    case DeleteFromTable(r: DataSourceV2Relation, optionalCond @ Some(cond)) =>

Review comment:
       @dilipbiswal, this can be reverted as well.




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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I have the rule locally, @dilipbiswal @rdblue. Adding some tests and will submit a 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.

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       Thanks for pointing me to the code! Looks like I was looking into it at the time you were writing this, which is why my comment below was just a bit later. I think we're all on the same page 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.

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeInto.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.plans.logical
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions.col
+
+case class MergeInto(mergeIntoProcessor: MergeIntoProcessor,
+                     targetRelation: DataSourceV2Relation,
+                     child: LogicalPlan) extends UnaryNode {
+  override def output: Seq[Attribute] = targetRelation.output
+}
+
+class MergeIntoProcessor(isSourceRowNotPresent: Expression,
+                         isTargetRowNotPresent: Expression,
+                         matchedConditions: Seq[Expression],
+                         matchedOutputs: Seq[Seq[Expression]],
+                         notMatchedConditions: Seq[Expression],
+                         notMatchedOutputs: Seq[Seq[Expression]],
+                         targetOutput: Seq[Expression],
+                         joinedAttributes: Seq[Attribute]) extends Serializable {

Review comment:
       This is essentially a physical plan node that is linked into both the physical plan and logical plan. I think it should be a normal physical plan node that is created in a strategy, just like other plans.
   
   The main issue with the way this PR currently works is that it doesn't delegate enough to the rest of the Spark planner. All of the analysis is done during rewrite in the optimizer, for example. I think that this should be broken up into analysis rules to validate and update the `MergeInto` plan, the rewrite rule to build the optimizations and join, and a strategy to convert the logical plan into a `MergeIntoExec`. I think this should also have a validation rule that checks each action to ensure that the expressions for that action are correctly 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.

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))
+
+    scan match {
+      case filterable: SupportsFileFilter =>
+        val matchingFilePlan = buildFileFilterPlan(prunedTargetPlan)
+        val dynamicFileFilter = DynamicFileFilter(scanRelation, matchingFilePlan, filterable)
+        dynamicFileFilter
+      case _ =>
+        scanRelation
+    }
+  }
+
+  private def buildFileFilterPlan(prunedTargetPlan: LogicalPlan): LogicalPlan = {

Review comment:
       @rdblue We will have issue if target has a column named `_file` as well, right ? I will mark it as a TODO and address in a follow-up, if its okay ? Need to think how to disambiguate this .. probably though a alias which checks for presence of the column and assigns a unique alias ?




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

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



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


[GitHub] [iceberg] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @aokolnychyi Hey Anton, sorry, i haven't had. chance to work on this in last couple of days. I will be looking at it from tomorrow/wednesday. 
   
   Firstly. i like the option of making the count check optional. In our use case mostly we will keep the count check off as we strictly control the merge statement we issue.
   
   About count implementation Anton, i was thinking to implement it without the optimization as a first cut and optimize it in a follow-up. The reason is, implementing the "first-cut" will not take much time. So all the time we will spend is to implement the follow-up pr to optimize.  That way, we can discuss the approaches in a targeted fashion in that PR. wdyt ?


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
##########
@@ -103,6 +103,7 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
   }
 
   private def buildFileFilterPlan(matchingRowsPlan: LogicalPlan): LogicalPlan = {
+    // TODO: For merge-into make sure _file is resolved only from target table.

Review comment:
       You can solve this problem by passing the target table attrs from the `DataSourceV2ScanRelation`:
   
   ```scala
       val matchingFilePlan = buildFileFilterPlan(scanRelation.output, matchingRowsPlanBuilder(scanRelation))
     ...
   
     private def buildFileFilterPlan(tableAttrs: Seq[AttributeReference], matchingRowsPlan: LogicalPlan): LogicalPlan = {
       val fileAttr = findOutputAttr(tableAttrs, FILE_NAME_COL)
       val agg = Aggregate(Seq(fileAttr), Seq(fileAttr), matchingRowsPlan)
       Project(Seq(findOutputAttr(agg.output, FILE_NAME_COL)), agg)
     }
   
     protected def findOutputAttr(attrs: Seq[Attribute], attrName: String): Attribute = {
       attrs.find(attr => resolver(attr.name, attrName)).getOrElse {
         throw new AnalysisException(s"Cannot find $attrName in $attrs")
       }
     }
   ```




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

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



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


[GitHub] [iceberg] dilipbiswal edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

Posted by GitBox <gi...@apache.org>.
dilipbiswal edited a comment on pull request #1947:
URL: https://github.com/apache/iceberg/pull/1947#issuecomment-753839534


   @aokolnychyi @rdblue 
   I have cleaned up the PR to address the comments. This is the base PR that has core logic for MERGE INTO processing. I have opened two separate PRs 1) Cardinality check 2) Grouping and sorting rows before write. We can discuss any issues with the implementation of (1) or (2) separately in those PRs. Hope it is okay ?
   
   Here are the two PRs.
   https://github.com/apache/iceberg/pull/2021
   https://github.com/apache/iceberg/pull/2022
   
   Thanks
   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records
+      case MergeIntoTable(target: DataSourceV2Relation,
+                          source: LogicalPlan, cond, actions, notActions) =>
+        val targetOutputCols = target.output
+        val newProjectCols = target.output ++ Seq(Alias(InputFileName(), FILE_NAME_COL)())
+        val newTargetTable = Project(newProjectCols, target)
+
+        // Construct the plan to prune target based on join condition between source and
+        // target.
+        val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE)
+        val writeInfo = newWriteInfo(target.schema)
+        val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo)

Review comment:
       The operation passed to the merge builder should be `merge`. And we will want to add tests that the isolation level is carried through correctly.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,

Review comment:
       Looks like this variable name wasn't updated when the processor was renamed to params.




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   Thanks @dilipbiswal! I'll take a closer look at this tomorrow.


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

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 pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > Skewness of partitions is an orthogonal problem and not specific to MERGE INTO , am i right ?
   
   The full outer join probably requires shuffling data, which means that it will be distributed by the MATCH expression. There's no guarantee that the match expression is aligned with the table partitioning. If it isn't, then writing without a sort would introduce a ton of small files because each task would be writing to each output partition.
   
   To avoid the small files problem, we need to repartition. If we repartition by just the partition expressions from the table, there is a good chance of producing a plan with too few tasks in the write stage because Spark can't split tasks for the same key. This is what introduces the skew. To avoid that, we can use a global sort to plan tasks that are balanced.
   
   A global sort is a best practice for writing anyway because it clusters data for faster reads.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/RewriteRowLevelOperationHelper.scala
##########
@@ -54,12 +54,12 @@ trait RewriteRowLevelOperationHelper extends PredicateHelper with Logging {
       table: Table,
       tableAttrs: Seq[AttributeReference],
       mergeBuilder: MergeBuilder,
-      cond: Expression,
+      cond: Option[Expression] = None,
       matchingRowsPlanBuilder: DataSourceV2ScanRelation => LogicalPlan): LogicalPlan = {
 
     val scanBuilder = mergeBuilder.asScanBuilder
 
-    pushFilters(scanBuilder, cond, tableAttrs)
+    cond.map(pushFilters(scanBuilder, _, tableAttrs))

Review comment:
       I don't think this change is needed because Anton's update already extracts the correct filters from `cond`.




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

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] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala
##########
@@ -43,6 +43,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) {
     // TODO: PullupCorrelatedPredicates should handle row-level operations
     extensions.injectOptimizerRule { _ => PullupCorrelatedPredicatesInRowLevelOperations }
     extensions.injectOptimizerRule { _ => RewriteDelete }
+    extensions.injectOptimizerRule { _ => RewriteMergeInto }

Review comment:
       @aokolnychyi will address this with a follow-up to not allow subqs and add tests to verify the same. Hope its okay.




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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   I think this PR is a great start, @dilipbiswal!
   
   I noted the following points that we need to address for correctness (some may be done separately):
   - We should perform the cardinality check as SQL standard requires.
   - We should align the assignments according to the target columns. This also applies to UPDATE.
   - We should think about how to group data before writing.
   - We will need more tests.
   
   There are also good to have points (can be done in follow-ups if too much trouble):
   - We should use a LEFT ANTI join for merge statements with only `WHEN NOT MATCHED THEN INSERT` clause.
   - We should use a RIGHT OUTER join for merge statements with only `WHEN MATCHED` clauses.
   
   Let's discuss each point one by one.
   
   **Cardinality Check**
   
   SQL standard requires an exception to be thrown if the ON clause in MERGE is such that more than 1 row in source matches a row in target. See [this](https://issues.apache.org/jira/browse/HIVE-14949) Hive issue for more info.
   
   Some systems do the cardinality check all the time while some, like Hive, make it optional. I'd say we should make it optional and let users configure it in the table properties by adding `write.merge.cardinality-check.enabled` property (true by default). The main problem with the cardinality check is the performance penalty: it requires an inner join. We are already doing this inner join for copy-on-write to find matches so we can modify that code to also do the cardinality check at the same time. I don't think we need an inner join for merge-on-read, though. 
   
   To sum up, I'd vote for having a flag in table properties and make the cardinality check optional (just like Hive ACID).
   
   We need to think a bit about how we implement the cardinality check. Here, I am open to suggestions. One idea is to modify nodes for dynamic file filtering. For example, we can use `monotonically_increasing_id` until we have `row_id` metadata column, append it to rows in the target table before the inner join to find matches and then perform the cardinality check and collect matching files. In order to make this efficient, we should reuse as much work as possible.
   
   One way to do that is to leverage an accumulator to track matching files:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - define a udf that accepts the file name, adds it to the accumulator and retuns 1
   - group by `_row_id`, perform the cardinality check
   - access the accumulator to get the matching files
   
   Another way is like this:
   - append `_row_id` and `_file` columns to the target table
   - do an inner join on the merge condition
   - select `_row_id`,  `_file` into a separate plan
   - temporarily cache the created plan (or persist it on executor nodes)
   - perform 2 queries: one for cardinality check and one for the matching files
   
   **Align Assignments**
   
   I don't think Spark aligns the assignments inside UPDATE or MERGE. We won't be able to support updating nested fields without it. We will probably need a separate rule for this. The same rule can be applied to UPDATE.
   
   **Group Data before Writing**
   
   We need to think about how to group data before writing new files with our updates and new records. One option is to group and order by partition columns. Another option is to group and order by the sort spec. The third option is to group updates and new records separately. 


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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanHelper.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.utils
+
+import java.util.UUID
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, PredicateHelper}
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DynamicFileFilter, LogicalPlan}
+import org.apache.spark.sql.connector.catalog.Table
+import org.apache.spark.sql.connector.iceberg.read.SupportsFileFilter
+import org.apache.spark.sql.connector.iceberg.write.MergeBuilder
+import org.apache.spark.sql.connector.write.{LogicalWriteInfo, LogicalWriteInfoImpl}
+import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanRelation}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+trait PlanHelper extends PredicateHelper {
+  val FILE_NAME_COL = "_file"
+  val ROW_POS_COL = "_pos"
+
+  def buildScanPlan(table: Table,
+                    output: Seq[AttributeReference],
+                    mergeBuilder: MergeBuilder,
+                    prunedTargetPlan: LogicalPlan): LogicalPlan = {
+
+    val scanBuilder = mergeBuilder.asScanBuilder
+    val scan = scanBuilder.build()
+    val scanRelation = DataSourceV2ScanRelation(table, scan, toOutputAttrs(scan.readSchema(), output))
+
+    scan match {
+      case filterable: SupportsFileFilter =>
+        val matchingFilePlan = buildFileFilterPlan(prunedTargetPlan)
+        val dynamicFileFilter = DynamicFileFilter(scanRelation, matchingFilePlan, filterable)
+        dynamicFileFilter
+      case _ =>
+        scanRelation
+    }
+  }
+
+  private def buildFileFilterPlan(prunedTargetPlan: LogicalPlan): LogicalPlan = {
+    val fileAttr = findOutputAttr(prunedTargetPlan, FILE_NAME_COL)
+    Aggregate(Seq(fileAttr), Seq(fileAttr), prunedTargetPlan)
+  }
+
+  def findOutputAttr(plan: LogicalPlan, attrName: String): Attribute = {
+    val resolver = SQLConf.get.resolver

Review comment:
       We should probably pass it from the session like in `AlignMergeIntoTable`.




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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"
+  val ROW_FROM_TARGET = "_row_from_target_"
+
+  import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._
+
+  override def apply(plan: LogicalPlan): LogicalPlan = {
+    plan resolveOperators {
+      // rewrite all operations that require reading the table to delete records

Review comment:
       @aokolnychyi cut-paste.. sorry, will 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.

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] dilipbiswal commented on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   @rdblue @aokolnychyi 
   I am still coming to speed on the comments on Grouping and sorting the data before writing. I think repartitioning and sorting the data within each partition (local sort) is the most performant one ? Skewness of partitions is an orthogonal problem and not specific to MERGE INTO , am i right ? 
   
   Ryan/Anton, can you tell me what do we do in terms of partitioning and sorting for CTAS and INSERT ... INTO SELECT FROM .. case today ? 
   
   
   


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

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 #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoProcessor
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoProcessor,

Review comment:
       I think that `MergeIntoProcessor` and this node should be merged. That's really a physical plan node and it is strange how it is created and passed through the logical plan.




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)
+
+    def shouldDeleteRow(row: InternalRow): Boolean =
+      row.getBoolean(params.targetOutput.size - 1)
+
+
+    def processRow(inputRow: InternalRow): InternalRow = {
+      isSourceRowNotPresentPred.eval(inputRow) match {

Review comment:
       Why use a `match` here and not `if (isSourceRowNotPresentPred.eval(inputRow))`?
   
   If you did that, it would be a bit cleaner:
   
   ```scala
         if (isSourceRowNotPresentPred.eval(inputRow)) {
           projectTargetCols.apply(inputRow)
         } else if (isTargetRowNotPresentPred.eval(inputRow)) {
           applyProjection(notMatchedPreds, notMatchedProjs, projectTargetCols,
             projectDeletedRow, inputRow, true)
         } else {
           applyProjection(matchedPreds, matchedProjs, projectTargetCols,
             projectDeletedRow, inputRow, false)
         }
   ```




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

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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)
+        } else {
+          projectTargetCols.apply(inputRow)
+        }
+    }
+  }
+
+  def processPartition(params: MergeIntoParams,
+                       rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = {
+    val joinedAttrs = params.joinedAttributes
+    val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs)
+    val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs)
+    val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs))
+    val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs))
+    val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs))
+    val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs))
+    val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs)
+    val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs)

Review comment:
       These last two projections are only needed when `notMatchedPreds` or `matchedPreds` does not have a default case, i.e. `lit(true)`.
   
   In the rewrite, there is also a function, `getClauseCondition`, that fills in `lit(true)` if there is no clause condition. But I don't think that any predicates _after_ the `true` condition are dropped.
   
   I think we could simplify the logic here and avoid extra clauses by ensuring that both `matchedPreds` and `notMatchedPreds` end with `lit(true)`. Then this class would not need to account for the case where no predicate matches and we wouldn't have extra predicates passed through. Last, we wouldn't need the last two projections here or in `MergeIntoParams` because they would be added to `notMatchedProjs` or `matchedProjs`.




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

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



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


[GitHub] [iceberg] aokolnychyi edited a comment on pull request #1947: [WIP] Spark MERGE INTO Support (copy-on-write implementation)

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


   > Please take a look at #1955. That exposes _pos so that we can use it.
   
   That is a great PR, let's get it in today.
   
   > If the table has a sort order, add a global sort of all the rows produced by the merge node.
   
   We have this option internally and it works well in some cases. There are a few things we need to be careful about, though.
   
   First, Spark will do a skew estimation step and the actual shuffle using two separate jobs. We don't want to recompute the merge join twice. Internally, we add a repartition stage after the join if a global sort on write is requested. While it does help a bit, it is not ideal. We have seen cases where the sort on write is by far the most expensive step of MERGE.
   
   Second, even when we do a global sort, the layout within partitions won't be ideal. So people will most likely have to compact again making the global sort during MERGE redundant.
   
   That's why we have to be careful about a global sort by default. I think this ultimately depends on the use case. Shall we make this configurable in table properties? How many query engines will follow it? Should that config be `copy-on-write` specific? I don't have answers to all the questions but it sounds reasonable to explore.
   
   At the same time, if we don't do the global sort, we may end up having too many small files after the operation. We can consider doing a repartition by the partition columns and sorting by the sort key but that will suffer if we have a lot of data for a single partition. It would be great to know the number of files and the size of data we need to rewrite per partition to make a good decision here.
   
   To sum up this case,
   
   - global sort -> good but expensive
   - repartition and then sort -> less expensive but what if too much data per partition?
   
   > If the table does not have a sort order, then add a default sort by _file, _pos, partition columns, and the MERGE condition's references.
   
   Sorting updated records by `_file` and `_pos` may be a bit tricky. For example, I have a file with columns (p, c1, c2, c3) in partition 'A' that is sorted by c1 and c2. If I have a merge command that updates c2 column (part of my sort key), my new records will be probably out of order if I sort by `_file` and `_pos`. That said, this is a fallback scenario so it may be not that big a deal. 
   
   


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

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



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


[GitHub] [iceberg] dilipbiswal commented on a change in pull request #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeIntoExec.scala
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, BasePredicate, Expression, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+
+case class MergeIntoExec(mergeIntoProcessor: MergeIntoParams,
+                         @transient targetRelation: DataSourceV2Relation,
+                         override val child: SparkPlan) extends UnaryExecNode {
+
+  override def output: Seq[Attribute] = targetRelation.output
+
+  protected override def doExecute(): RDD[InternalRow] = {
+    child.execute().mapPartitions {
+      processPartition(mergeIntoProcessor, _)
+    }
+  }
+
+  private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = {
+    UnsafeProjection.create(exprs, attrs)
+  }
+
+  private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = {
+    GeneratePredicate.generate(expr, attrs)
+  }
+
+  def applyProjection(predicates: Seq[BasePredicate],
+                      projections: Seq[UnsafeProjection],
+                      projectTargetCols: UnsafeProjection,
+                      projectDeleteRow: UnsafeProjection,
+                      inputRow: InternalRow,
+                      targetRowNotPresent: Boolean): InternalRow = {
+    // Find the first combination where the predicate evaluates to true
+    val pair = (predicates zip projections).find {
+      case (predicate, _) => predicate.eval(inputRow)
+    }
+
+    // Now apply the appropriate projection to either :
+    // - Insert a row into target
+    // - Update a row of target
+    // - Delete a row in target. The projected row will have the deleted bit set.
+    pair match {
+      case Some((_, projection)) =>
+        projection.apply(inputRow)
+      case None =>
+        if (targetRowNotPresent) {
+          projectDeleteRow.apply(inputRow)

Review comment:
       @rdblue I had thought about it. But couldn't think of a way to do it. How about, we create a materialized delete row once per partition like this :
   ```
   val deleteExpr = params.targetOutput.dropRight(1).map(e => Literal.default(e.dataType)) ++ Seq(Literal.create(true, BooleanType))
       val deletedRow1 = UnsafeProjection.create(deleteExpr)
       val deletedRow = deletedRow1.apply(null)
   ```
   deteExpr will come from rewriteMergeInto just like its passed now. Here we will just create the InternalRow once and use it ? Will that work Ryan ?   




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: 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 #1947: Spark MERGE INTO Support (copy-on-write implementation)

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



##########
File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala
##########
@@ -0,0 +1,103 @@
+/*
+ * 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.internal.Logging
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner}
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.utils.PlanHelper
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.internal.SQLConf
+
+object RewriteMergeInto extends Rule[LogicalPlan] with PlanHelper with Logging  {
+  val ROW_FROM_SOURCE = "_row_from_source_"

Review comment:
       Shall we make these variables private?




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

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



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