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

[GitHub] [iceberg] aokolnychyi opened a new pull request #3763: Spark: Implement merge-on-read DELETE

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


   This PR implements merge-on-read DELETE in Spark.
   
   Resolves #3629.


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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,46 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {
+    createAndInitTable("id INT, dep STRING, category STRING");
+
+    // write an unpartitioned file
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");
+
+    // write a file partitioned by dep
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+    append(tableName,
+        "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\" }\n" +
+            "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }");
+
+    // write a file partitioned by dep and category
+    sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName);
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");

Review comment:
       This test would be a lot more readable if the rows were unique rather than reusing id=1 in several 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();

Review comment:
       @rdblue, it can be cleaner to replace these two methods with a single method like in `OverwriteFiles`. I don't see a use case where we would want to set one but not the other. Unfortunately, we have released this in 0.12.1 so I am not it is worth the effort now.
   
   We have the following methods in `OverwriteFiles`:
   
   ```
   validateNoConflictingData()
   validateNoConflictingDeletes()
   ```




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask {
+
+  override protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit = {
+    val deltaWriter = writer.asInstanceOf[DeltaWriter[InternalRow]]
+    val operation = row.getInt(0)
+    val rowProjection = projs.rowProjection.orNull
+    val rowIdProjection = projs.rowIdProjection
+    val metadataProjection = projs.metadataProjection.orNull

Review comment:
       Can these be lazy class vals instead? That would avoid needing to access them for every row.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {

Review comment:
       I think it would be better to make this `static` and pass in `scan`. That's cleaner because right now this requires being wrapped in a check that `scan` is not null. Instead of having methods that require certain states for fields, I prefer to have static methods that make their args explicit. That is, if you pass in `scan` you see that it should be non-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.

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",

Review comment:
       Does this mean `no validation required`? I think it would be good to note that it isn't required, rather than leaving the confusing possibility that it was skipped.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/ProjectingInternalRow.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class ProjectingInternalRow(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       I mean you could create a `GenericInternalRow` by creating one and filling it with the values from the existing row. I'm not sure what's more expensive, but it probably doesn't matter since I doubt you're actually calling this method anywhere.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       Why not resolve the attrs against the child `query: Projection`? That's where the data is coming from. So you'd be finding the row ID fields that are coming from the incoming data that will be extracted by `projections.rowIdProjection`.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();

Review comment:
       I'd say let's keep the API as it is since it's already released.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -61,98 +63,32 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkBatchQueryScan.class);
 
-  private final Long snapshotId;
-  private final Long startSnapshotId;
-  private final Long endSnapshotId;
-  private final Long asOfTimestamp;
-  private final Long splitSize;
-  private final Integer splitLookback;
-  private final Long splitOpenFileCost;
+  private final TableScan scan;
+  private final Context ctx;
   private final List<Expression> runtimeFilterExpressions;
 
-  private TableScan tableScan = null; // lazy scan
   private Set<Integer> specIds = null; // lazy cache of scanned spec IDs
   private List<FileScanTask> files = null; // lazy cache of files
   private List<CombinedScanTask> tasks = null; // lazy cache of tasks
 
-  SparkBatchQueryScan(SparkSession spark, Table table, SparkReadConf readConf,
-                      Schema expectedSchema, List<Expression> filters) {
+  SparkBatchQueryScan(SparkSession spark, Table table, TableScan scan, Context ctx,
+                      SparkReadConf readConf, Schema expectedSchema, List<Expression> filters) {
 
     super(spark, table, readConf, expectedSchema, filters);
 
-    this.snapshotId = readConf.snapshotId();

Review comment:
       Moved to `SparkScanBuilder`.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // TODO: validate the row ID and metadata schema

Review comment:
       I'm debating this. It seems like it will always be correct, but resolution is a nice way to sanity check and fail rather than doing the wrong thing at runtime.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/util/StructProjection.java
##########
@@ -58,6 +58,19 @@ public static StructProjection create(Schema dataSchema, Schema projectedSchema)
     return new StructProjection(dataSchema.asStruct(), projectedSchema.asStruct());
   }
 
+  /**
+   * Creates a projecting wrapper for {@link StructLike} rows.
+   * <p>
+   * This projection does not work with repeated types like lists and maps.
+   *
+   * @param structType type of rows wrapped by this projection
+   * @param projectedStructType result type of the projected rows
+   * @return a wrapper to project rows
+   */
+  public static StructProjection create(StructType structType, StructType projectedStructType) {
+    return new StructProjection(structType, projectedStructType, false);

Review comment:
       Can this use the constructor that defaults `false` instead of adding a default here? That way refactoring later is easier.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/ProjectingInternalRow.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class ProjectingInternalRow(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       Assuming it is an `UnsafeRow` passed in, you might be right. I'd just leave this as is.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -310,4 +263,62 @@ public String toString() {
         "IcebergScan(table=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)",
         table(), expectedSchema().asStruct(), filterExpressions(), runtimeFilterExpressions, caseSensitive());
   }
+
+  static class Context {

Review comment:
       Seems like initializing `SparkReadConf` with a specific snapshot would work then?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       I thought about this too but it seems such a sensitive area that gets invoked for every row so I've tried to avoid any extra actions. While try/catch does not cost much unless there is an exception, JVM may not rewrite and perform advanced optimizations on the code inside the block. And having an extra if for would potentially be even worse.
   
   I double checked the code that produces these projections and it seems unlikely we can get an NPE here given our tests.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -319,20 +319,45 @@ public static boolean isPromotionAllowed(Type from, Type.PrimitiveType to) {
    */
   public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
                                          Boolean checkNullability, Boolean checkOrdering) {
+    String errMsg = "Cannot write incompatible dataset to table with schema:";
+    checkSchemaCompatibility(errMsg, tableSchema, writeSchema, checkNullability, checkOrdering);
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param context the schema context (e.g. row ID)
+   * @param expectedSchema the expected schema
+   * @param providedSchema the provided schema
+   * @param checkNullability whether to check field nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String context, Schema expectedSchema, Schema providedSchema,
+                                    boolean checkNullability, boolean checkOrdering) {
+    String errMsg = String.format("Provided %s schema is incompatible with expected %s schema:", context, context);
+    checkSchemaCompatibility(errMsg, expectedSchema, providedSchema, checkNullability, checkOrdering);
+  }
+
+  private static void checkSchemaCompatibility(String errMsg, Schema schema, Schema providedSchema,
+                                               boolean checkNullability, boolean checkOrdering) {
     List<String> errors;
     if (checkNullability) {
-      errors = CheckCompatibility.writeCompatibilityErrors(tableSchema, writeSchema, checkOrdering);
+      errors = CheckCompatibility.writeCompatibilityErrors(schema, providedSchema, checkOrdering);
     } else {
-      errors = CheckCompatibility.typeCompatibilityErrors(tableSchema, writeSchema, checkOrdering);
+      errors = CheckCompatibility.typeCompatibilityErrors(schema, providedSchema, checkOrdering);
     }
 
     if (!errors.isEmpty()) {
       StringBuilder sb = new StringBuilder();
-      sb.append("Cannot write incompatible dataset to table with schema:\n")
-          .append(tableSchema)
-          .append("\nwrite schema:")
-          .append(writeSchema)
-          .append("\nProblems:");
+      sb.append(errMsg)
+          .append("\n")
+          .append(schema)
+          .append("\n")
+          .append("provided schema:")
+          .append("\n")
+          .append(providedSchema)
+          .append("\n")
+          .append("problems:");

Review comment:
       We capitalized only one before so it looked inconsistent. Made both start with capital letters now.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -157,8 +159,73 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
+    Long snapshotId = readConf.snapshotId();
+    Long asOfTimestamp = readConf.asOfTimestamp();
+
+    Preconditions.checkArgument(snapshotId == null || asOfTimestamp == null,
+        "Cannot set both %s and %s to select which table snapshot to scan",
+        SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+
+    Long startSnapshotId = readConf.startSnapshotId();
+    Long endSnapshotId = readConf.endSnapshotId();
+
+    if (snapshotId != null || asOfTimestamp != null) {
+      Preconditions.checkArgument(startSnapshotId == null && endSnapshotId == null,
+          "Cannot set %s and %s for incremental scans when either %s or %s is set",
+          SparkReadOptions.START_SNAPSHOT_ID, SparkReadOptions.END_SNAPSHOT_ID,
+          SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+    }
+
+    Preconditions.checkArgument(startSnapshotId != null || endSnapshotId == null,
+        "Cannot set only %s for incremental scans. Please, set %s too.",
+        SparkReadOptions.END_SNAPSHOT_ID, SparkReadOptions.START_SNAPSHOT_ID);
+
+    SparkBatchQueryScan.Context ctx = new SparkBatchQueryScan.Context(
+        snapshotId, asOfTimestamp,
+        startSnapshotId, endSnapshotId);
+
+    return buildBatchQueryScan(ctx);
+  }
+
+  public Scan buildMergeOnReadScan() {
+    Snapshot snapshot = table.currentSnapshot();

Review comment:
       Added the validation.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,48 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {

Review comment:
       Looks good to me!




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       I thought about this too but it seems such a sensitive area that gets invoked for every row so I've tried to avoid any extra actions. While try/catch does not cost much unless there is an exception, JVM may not rewrite and perform advanced optimizations on the code inside the block. And having an extra if for would potentially be even worse.
   
   I checked the code that produces these projections and it seems unlikely we can get an NPE given our tests.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java
##########
@@ -231,6 +247,31 @@ public DistributionMode copyOnWriteMergeDistributionMode() {
     }
   }
 
+  public DistributionMode positionDeleteDistributionMode() {

Review comment:
       Seems like some reasonable defaults to me.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -61,98 +63,32 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkBatchQueryScan.class);
 
-  private final Long snapshotId;
-  private final Long startSnapshotId;
-  private final Long endSnapshotId;
-  private final Long asOfTimestamp;
-  private final Long splitSize;
-  private final Integer splitLookback;
-  private final Long splitOpenFileCost;
+  private final TableScan scan;
+  private final Context ctx;

Review comment:
       I am not a big fan of this class here but it is needed for `equals` and `hashCode`. Another option I considered was to implement `equals` and `hashCode` in all `TableScan` implementations. Unfortunately, we have a lot of such classes and "equal" scans in Spark are a slightly weaker concept (i.e. not every detail must be the same to consider two scans identical).
   
   Alternatives are welcome.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask {
+
+  override protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit = {
+    val deltaWriter = writer.asInstanceOf[DeltaWriter[InternalRow]]
+    val operation = row.getInt(0)
+    val rowProjection = projs.rowProjection.orNull
+    val rowIdProjection = projs.rowIdProjection
+    val metadataProjection = projs.metadataProjection.orNull

Review comment:
       Oops, that's old. Fixed.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);
+  }
+
+  private Schema expectedMetadataSchema() {

Review comment:
       I got rid of this method.
   
   I also like pure functions but I am not a big fan of using static methods unless we need to. For example, static functions should probably be groped together. I, on the other hand, prefer grouping by logic. Also, we sometimes have too many arguments to pass and method look simpler with instance variables (not in this case). However, +1 to having pure functions whenever possible.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/ProjectingInternalRow.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class ProjectingInternalRow(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       Yeah, I'd do that too for now.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);

Review comment:
       Looks like this could be a static final field.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {

Review comment:
       I think this is a bit confusing so it warrants a comment. If the snapshot ID is null, then this validates back to the start of the table.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -340,6 +340,42 @@ public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
     }
   }
 
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param ctx the validation context used in the error message
+   * @param expectedSchema the expected schema
+   * @param actualSchema the actual schema
+   * @param checkNullability whether to check nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String ctx, Schema expectedSchema, Schema actualSchema,
+                                    Boolean checkNullability, Boolean checkOrdering) {
+    List<String> errors;
+    if (checkNullability) {
+      errors = CheckCompatibility.writeCompatibilityErrors(expectedSchema, actualSchema, checkOrdering);
+    } else {
+      errors = CheckCompatibility.typeCompatibilityErrors(expectedSchema, actualSchema, checkOrdering);
+    }
+
+    if (!errors.isEmpty()) {
+      StringBuilder sb = new StringBuilder();
+      String header = String.format("Schema is not compatible with expected %s schema:", ctx);
+      sb.append(header)
+          .append("\nexpected schema:")

Review comment:
       I don't think this is needed since the last line ends with `expected %s schema:`




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);

Review comment:
       Nit: prefer full words, like `context` over `ctx`. They're usually no harder to type since dropping vowels is awkward. And it helps readability.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -54,6 +60,50 @@ trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
     }
   }
 
+  protected def buildWriteDeltaProjections(
+      plan: LogicalPlan,
+      rowAttrs: Seq[Attribute],
+      rowIdAttrs: Seq[Attribute],
+      metadataAttrs: Seq[Attribute]): WriteDeltaProjections = {
+
+    val rowProjection = if (rowAttrs.nonEmpty) {
+      Some(newProjection(plan, rowAttrs, usePlanTypes = true))
+    } else {
+      None
+    }
+
+    // in MERGE, the plan may contain both delete and insert records that may affect
+    // the nullability of metadata columns (e.g. metadata columns for new records are always null)
+    // since metadata columns are never passed with new records to insert,
+    // use the actual metadata column type instead of the one present in the plan
+
+    val rowIdProjection = newProjection(plan, rowIdAttrs, usePlanTypes = false)
+
+    val metadataProjection = if (metadataAttrs.nonEmpty) {
+      Some(newProjection(plan, metadataAttrs, usePlanTypes = false))
+    } else {
+      None
+    }
+
+    WriteDeltaProjections(rowProjection, rowIdProjection, metadataProjection)
+  }
+
+  // the projection is done by name, ignoring expr IDs
+  private def newProjection(

Review comment:
       It would be good to distinguish what this is doing from a `Project` node. What about calling this a lazy projection?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -157,8 +159,73 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
+    Long snapshotId = readConf.snapshotId();
+    Long asOfTimestamp = readConf.asOfTimestamp();
+
+    Preconditions.checkArgument(snapshotId == null || asOfTimestamp == null,
+        "Cannot set both %s and %s to select which table snapshot to scan",
+        SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+
+    Long startSnapshotId = readConf.startSnapshotId();
+    Long endSnapshotId = readConf.endSnapshotId();
+
+    if (snapshotId != null || asOfTimestamp != null) {
+      Preconditions.checkArgument(startSnapshotId == null && endSnapshotId == null,

Review comment:
       Can these checks be moved into `SparkReadConf` as a `validate` 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)

Review comment:
       Added a cast here.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -310,4 +263,62 @@ public String toString() {
         "IcebergScan(table=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)",
         table(), expectedSchema().asStruct(), filterExpressions(), runtimeFilterExpressions, caseSensitive());
   }
+
+  static class Context {

Review comment:
       Like I wrote [here](https://github.com/apache/iceberg/pull/3763#discussion_r771475824), I am not happy with this place either.
   
   For merge-on-read scans, the snapshot ID does not come from `SparkReadConf`. Instead, it comes from the current table snapshot no matter what. `SparkScanBuilder` has more context. Let me think more about this.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/util/StructProjection.java
##########
@@ -58,6 +58,19 @@ public static StructProjection create(Schema dataSchema, Schema projectedSchema)
     return new StructProjection(dataSchema.asStruct(), projectedSchema.asStruct());
   }
 
+  /**
+   * Creates a projecting wrapper for {@link StructLike} rows.
+   * <p>
+   * This projection does not work with repeated types like lists and maps.
+   *
+   * @param structType type of rows wrapped by this projection
+   * @param projectedStructType result type of the projected rows
+   * @return a wrapper to project rows
+   */
+  public static StructProjection create(StructType structType, StructType projectedStructType) {
+    return new StructProjection(structType, projectedStructType, false);

Review comment:
       Fixed.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {

Review comment:
       I added some comments in the implementation.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       Yeah, I agree. I’ll try to implement. 




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,46 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {
+    createAndInitTable("id INT, dep STRING, category STRING");
+
+    // write an unpartitioned file
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");
+
+    // write a file partitioned by dep
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+    append(tableName,
+        "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\" }\n" +
+            "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }");
+
+    // write a file partitioned by dep and category
+    sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName);
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");

Review comment:
       My point is just that it doesn't need to be the same record in all places. That's why I suggested odds vs evens. If you use a different category and ID, then delete all the odd ones you would end up with the same test without the reader wondering why they all have the same ID.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {
+        case Some(outAttr) => isCompatible(field, outAttr)
+        case None => false
+      }
+    }
+  }
+
+  private def metadataAttrsResolved: Boolean = {
+    projections.metadataProjection match {
+      case Some(projection) =>
+        projection.schema.forall { field =>
+          originalTable.metadataOutput.exists(metadataAttr => isCompatible(field, metadataAttr))
+        }
+      case None => true
+    }
+  }
+
+  private def isCompatible(projectionField: StructField, outAttr: NamedExpression): Boolean = {

Review comment:
       Looks good.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       Why does this use `originalTable`? I thought these fields should be coming from `query`?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       That being said, it may not be the final iteration. Feedback would be appreciated.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       I think we cannot use `query` for MERGE commands. The actual nullability is defined by the projection and may differ from the nullability of the attributes in `query`. Consider a MERGE plan with records to update and insert. The metadata and row ID columns will be always nullable as those columns are null for records to insert. However, we never pass row ID or metadata columns with inserts. We only pass them with updates and deletes where those columns have correct values. In other words, the projection has more precise types. The existing logic checks that whatever the projection produces satisfies the original row ID and metadata 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #3763: Spark: Implement merge-on-read DELETE

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


   Thanks for reviewing, @rdblue! I've merged this one as the remaining open points are relatively minor and can be further discussed separately.


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {

Review comment:
       Requires extra attention!




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/InternalRowProjection.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class InternalRowProjection(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {

Review comment:
       Renamed.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {
+    private final Schema dataSchema;
+    private final StructType dataSparkType;
+    private final FileFormat dataFileFormat;
+    private final long targetDataFileSize;
+    private final StructType deleteSparkType;
+    private final StructType metadataSparkType;
+    private final FileFormat deleteFileFormat;
+    private final long targetDeleteFileSize;
+    private final boolean fanoutWriterEnabled;
+
+    Context(Schema dataSchema, SparkWriteConf writeConf, ExtendedLogicalWriteInfo info) {

Review comment:
       I am open to alternatives. The reason why I added it is to avoid passing 9 arguments separately. I usually try to avoid huge arg lists that need a few lines. That's a personal preference, though. What do you think, @rdblue?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",

Review comment:
       Fixed the commit message.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {

Review comment:
       @rdblue, what do you think of the validation here?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);

Review comment:
       Added a base writer for this.

##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec

Review comment:
       Added a comment.




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

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

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



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


[GitHub] [iceberg] aokolnychyi merged pull request #3763: Spark: Implement merge-on-read DELETE

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


   


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       @rdblue, I changed the validation a bit. Like discussed before, the intention is to validate whatever comes out of the projection satisfies the reported row ID attributes. I couldn't avoid using `originalTable` as the operation only tells me attribute names and I have to resolve them against something.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       I think we cannot use `query` for MERGE commands. The actual nullability is defined by the projection and may differ from the nullability of the attributes in `query`. Consider a MERGE plan with records to update and insert. The metadata and row ID columns will be always nullable as those columns are null for records to insert. However, we never pass row ID or metadata columns with inserts. We only pass them with updates and deletes where those columns have correct values. In other words, the projection has more precise types. The existing logic checks that whatever the projection produces satisfies the original row ID and metadata attrs.
   
   Apart from that, we still need `originalTable` to refresh the cache later.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       This may not be a good idea if we think we can guarantee that the required projections will be there.
   
   Maybe all we need instead is to catch NPE and wrap it with the projection context and operation.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -319,20 +319,45 @@ public static boolean isPromotionAllowed(Type from, Type.PrimitiveType to) {
    */
   public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
                                          Boolean checkNullability, Boolean checkOrdering) {
+    String errMsg = "Cannot write incompatible dataset to table with schema:";
+    checkSchemaCompatibility(errMsg, tableSchema, writeSchema, checkNullability, checkOrdering);
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param context the schema context (e.g. row ID)
+   * @param expectedSchema the expected schema
+   * @param providedSchema the provided schema
+   * @param checkNullability whether to check field nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String context, Schema expectedSchema, Schema providedSchema,
+                                    boolean checkNullability, boolean checkOrdering) {
+    String errMsg = String.format("Provided %s schema is incompatible with expected %s schema:", context, context);

Review comment:
       Nah, just one is probably enough. Updated.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(

Review comment:
       This is custom and needs reviews.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {

Review comment:
       Mostly copied from Spark.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/InternalRowProjection.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class InternalRowProjection(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       As long as there is a copy happening, why not build a new `GenericInternalRow`? Does it just not matter because of the limited use here?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -310,4 +263,62 @@ public String toString() {
         "IcebergScan(table=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)",
         table(), expectedSchema().asStruct(), filterExpressions(), runtimeFilterExpressions, caseSensitive());
   }
+
+  static class Context {

Review comment:
       Looks like this comes entirely from `SparkReadConf`, which is also passed into `SparkBatchQueryScan`. Instead of creating the context and passing it in, why not create it in this class? But then, if it is only used in this class, why not leave `equals` and `hashCode` unchanged and not use this at all?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       Well, it is a little bit tricky. The actual type is defined by the projection. For example, consider MERGE operations. The incoming plan will have wrong nullability for metadata and row ID columns (they will be always nullable as those columns are null for records to insert). However, we never pass row ID or metadata columns with inserts. We only pass them with updates and deletes where those columns have correct values. In other words, the projection has more precise types. The existing logic validates that whatever the projections produce satisfy the target output attributes.
   
   That being said, you are also right that we probably need some validation that we can actually project those columns from query...
   
   What do you think, @rdblue?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);
+  }
+
+  private Schema expectedMetadataSchema() {
+    NestedField partition = MetadataColumns.metadataColumn(table, MetadataColumns.PARTITION_COLUMN_NAME);

Review comment:
       Yeah, exactly. We need them to write position deletes to the correct spec/partition.

##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);

Review comment:
       Done.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -157,8 +159,73 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
+    Long snapshotId = readConf.snapshotId();
+    Long asOfTimestamp = readConf.asOfTimestamp();
+
+    Preconditions.checkArgument(snapshotId == null || asOfTimestamp == null,
+        "Cannot set both %s and %s to select which table snapshot to scan",
+        SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+
+    Long startSnapshotId = readConf.startSnapshotId();
+    Long endSnapshotId = readConf.endSnapshotId();
+
+    if (snapshotId != null || asOfTimestamp != null) {
+      Preconditions.checkArgument(startSnapshotId == null && endSnapshotId == null,

Review comment:
       I think it would be possible but would require adding validation capabilities to `SparkReadConf`. I think the current validation is easier to read than what we had before so I'd probably address this 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -336,7 +329,47 @@ public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
       for (String error : errors) {
         sb.append("\n* ").append(error);
       }
-      throw new IllegalArgumentException(sb.toString());
+      return sb.toString();
+    });
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param ctx the validation context used in the error message
+   * @param expectedSchema the expected schema
+   * @param actualSchema the actual schema
+   * @param checkNullability whether to check nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String ctx, Schema expectedSchema, Schema actualSchema,
+                                    Boolean checkNullability, Boolean checkOrdering) {
+    validateSchema(expectedSchema, actualSchema, checkNullability, checkOrdering, (errors) -> {
+      StringBuilder sb = new StringBuilder();
+      String header = String.format("Schema is not compatible with expected %s schema:", ctx);
+      sb.append(header)
+          .append(expectedSchema)
+          .append("\nactual schema:")
+          .append(actualSchema)
+          .append("\nproblems:");
+      for (String error : errors) {
+        sb.append("\n* ").append(error);
+      }
+      return sb.toString();
+    });
+  }
+
+  private static void validateSchema(Schema expectedSchema, Schema actualSchema, Boolean checkNullability,

Review comment:
       I did this refactoring to slightly reduce the code duplication. Now I look at it and I am not sure it was worth it.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java
##########
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Expressions;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperationInfo;
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final IsolationLevel isolationLevel;
+
+  // lazy vars
+  private ScanBuilder lazyScanBuilder;
+  private Scan configuredScan;
+  private DeltaWriteBuilder lazyWriteBuilder;
+
+  SparkPositionDeltaOperation(SparkSession spark, Table table, RowLevelOperationInfo info,
+                              IsolationLevel isolationLevel) {
+    this.spark = spark;
+    this.table = table;
+    this.command = info.command();
+    this.isolationLevel = isolationLevel;
+  }
+
+  @Override
+  public Command command() {
+    return command;
+  }
+
+  @Override
+  public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
+    if (lazyScanBuilder == null) {
+      this.lazyScanBuilder = new SparkScanBuilder(spark, table, options) {
+        @Override
+        public Scan build() {
+          Scan scan = super.buildMergeOnReadScan();
+          SparkPositionDeltaOperation.this.configuredScan = scan;
+          return scan;
+        }
+      };
+    }
+
+    return lazyScanBuilder;
+  }
+
+  @Override
+  public DeltaWriteBuilder newWriteBuilder(ExtendedLogicalWriteInfo info) {
+    if (lazyWriteBuilder == null) {
+      lazyWriteBuilder = new SparkPositionDeltaWriteBuilder(
+          spark, table, command, configuredScan,

Review comment:
       Added a comment.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {
+    private final Schema dataSchema;
+    private final StructType dataSparkType;
+    private final FileFormat dataFileFormat;
+    private final long targetDataFileSize;
+    private final StructType deleteSparkType;
+    private final StructType metadataSparkType;
+    private final FileFormat deleteFileFormat;
+    private final long targetDeleteFileSize;
+    private final boolean fanoutWriterEnabled;
+
+    Context(Schema dataSchema, SparkWriteConf writeConf, ExtendedLogicalWriteInfo info) {

Review comment:
       You are right it has to be serialized and `SparkWriteConf` isn't. That's probably why I added it.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {
+    private final Schema dataSchema;
+    private final StructType dataSparkType;
+    private final FileFormat dataFileFormat;
+    private final long targetDataFileSize;
+    private final StructType deleteSparkType;
+    private final StructType metadataSparkType;
+    private final FileFormat deleteFileFormat;
+    private final long targetDeleteFileSize;
+    private final boolean fanoutWriterEnabled;
+
+    Context(Schema dataSchema, SparkWriteConf writeConf, ExtendedLogicalWriteInfo info) {

Review comment:
       Added a comment at the top.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {

Review comment:
       Should this check the operation rather than the data schema?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java
##########
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Expressions;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperationInfo;
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final IsolationLevel isolationLevel;
+
+  // lazy vars
+  private ScanBuilder lazyScanBuilder;
+  private Scan configuredScan;
+  private DeltaWriteBuilder lazyWriteBuilder;
+
+  SparkPositionDeltaOperation(SparkSession spark, Table table, RowLevelOperationInfo info,
+                              IsolationLevel isolationLevel) {
+    this.spark = spark;
+    this.table = table;
+    this.command = info.command();
+    this.isolationLevel = isolationLevel;
+  }
+
+  @Override
+  public Command command() {
+    return command;
+  }
+
+  @Override
+  public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
+    if (lazyScanBuilder == null) {
+      this.lazyScanBuilder = new SparkScanBuilder(spark, table, options) {
+        @Override
+        public Scan build() {
+          Scan scan = super.buildMergeOnReadScan();
+          SparkPositionDeltaOperation.this.configuredScan = scan;
+          return scan;
+        }
+      };
+    }
+
+    return lazyScanBuilder;
+  }
+
+  @Override
+  public DeltaWriteBuilder newWriteBuilder(ExtendedLogicalWriteInfo info) {
+    if (lazyWriteBuilder == null) {
+      lazyWriteBuilder = new SparkPositionDeltaWriteBuilder(
+          spark, table, command, configuredScan,

Review comment:
       Assert that `configuredScan` is non-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.

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);

Review comment:
       Is it possible to create this in the writer instead of passing it all in?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,46 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {
+    createAndInitTable("id INT, dep STRING, category STRING");
+
+    // write an unpartitioned file
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");
+
+    // write a file partitioned by dep
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+    append(tableName,
+        "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\" }\n" +
+            "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }");
+
+    // write a file partitioned by dep and category
+    sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName);
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");

Review comment:
       Maybe use evens vs odds to 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.

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask {
+
+  override protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit = {
+    val deltaWriter = writer.asInstanceOf[DeltaWriter[InternalRow]]
+    val operation = row.getInt(0)
+    val rowProjection = projs.rowProjection.orNull
+    val rowIdProjection = projs.rowIdProjection
+    val metadataProjection = projs.metadataProjection.orNull
+
+    operation match {
+      case DELETE_OPERATION =>
+        rowIdProjection.project(row)
+        metadataProjection.project(row)
+        deltaWriter.delete(metadataProjection, rowIdProjection)

Review comment:
       Looks fine. Can you remind me why we pass the metadata in here?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -157,8 +159,73 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
+    Long snapshotId = readConf.snapshotId();
+    Long asOfTimestamp = readConf.asOfTimestamp();
+
+    Preconditions.checkArgument(snapshotId == null || asOfTimestamp == null,
+        "Cannot set both %s and %s to select which table snapshot to scan",
+        SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+
+    Long startSnapshotId = readConf.startSnapshotId();
+    Long endSnapshotId = readConf.endSnapshotId();
+
+    if (snapshotId != null || asOfTimestamp != null) {
+      Preconditions.checkArgument(startSnapshotId == null && endSnapshotId == null,
+          "Cannot set %s and %s for incremental scans when either %s or %s is set",
+          SparkReadOptions.START_SNAPSHOT_ID, SparkReadOptions.END_SNAPSHOT_ID,
+          SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+    }
+
+    Preconditions.checkArgument(startSnapshotId != null || endSnapshotId == null,
+        "Cannot set only %s for incremental scans. Please, set %s too.",
+        SparkReadOptions.END_SNAPSHOT_ID, SparkReadOptions.START_SNAPSHOT_ID);
+
+    SparkBatchQueryScan.Context ctx = new SparkBatchQueryScan.Context(
+        snapshotId, asOfTimestamp,
+        startSnapshotId, endSnapshotId);
+
+    return buildBatchQueryScan(ctx);
+  }
+
+  public Scan buildMergeOnReadScan() {
+    Snapshot snapshot = table.currentSnapshot();

Review comment:
       This still needs to validate that the snapshot is not set in SparkReadConf right?




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

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

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



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


[GitHub] [iceberg] rdblue commented on pull request #3763: Spark: Implement merge-on-read DELETE

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


   Still looks good to me. I had a few minor comments, but overall +1.


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/InternalRowProjection.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class InternalRowProjection(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       Replied [here](https://github.com/apache/iceberg/pull/3763/files#r773296971).




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkDistributionAndOrderingUtil.java
##########
@@ -96,6 +100,44 @@ public static Distribution buildCopyOnWriteDistribution(Table table, Command com
     }
   }
 
+  public static Distribution buildPositionDeltaDistribution(Table table, Command command,
+                                                            DistributionMode distributionMode) {
+    if (command == DELETE) {
+      return positionDeleteDistribution(distributionMode);
+    } else {
+      throw new IllegalArgumentException("Only position deletes are currently supported");
+    }
+  }
+
+  private static Distribution positionDeleteDistribution(DistributionMode distributionMode) {
+    switch (distributionMode) {
+      case NONE:
+        return Distributions.unspecified();
+
+      case HASH:
+        Expression[] clustering = new Expression[]{SPEC_ID, PARTITION};
+        return Distributions.clustered(clustering);
+
+      case RANGE:
+        SortOrder[] ordering = new SortOrder[]{SPEC_ID_ORDER, PARTITION_ORDER, FILE_PATH_ORDER};
+        return Distributions.ordered(ordering);
+
+      default:
+        throw new IllegalArgumentException("Unsupported distribution mode: " + distributionMode);
+    }
+  }
+
+  public static SortOrder[] buildPositionDeltaOrdering(Table table, Command command, Distribution distribution) {
+    // the spec requires position delete files to be sorted by file and pos
+    SortOrder[] deleteOrdering = new SortOrder[]{SPEC_ID_ORDER, PARTITION_ORDER, FILE_PATH_ORDER, ROW_POSITION_ORDER};

Review comment:
       It will be used in branches below once we support other commands but I removed for now.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java
##########
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.spark.source;
+
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.expressions.Expressions;
+import org.apache.spark.sql.connector.expressions.NamedReference;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperationInfo;
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.connector.read.ScanBuilder;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+
+class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final IsolationLevel isolationLevel;
+
+  // lazy vars
+  private ScanBuilder lazyScanBuilder;
+  private Scan configuredScan;
+  private DeltaWriteBuilder lazyWriteBuilder;
+
+  SparkPositionDeltaOperation(SparkSession spark, Table table, RowLevelOperationInfo info,
+                              IsolationLevel isolationLevel) {
+    this.spark = spark;
+    this.table = table;
+    this.command = info.command();
+    this.isolationLevel = isolationLevel;
+  }
+
+  @Override
+  public Command command() {
+    return command;
+  }
+
+  @Override
+  public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) {
+    if (lazyScanBuilder == null) {
+      this.lazyScanBuilder = new SparkScanBuilder(spark, table, options) {
+        @Override
+        public Scan build() {
+          Scan scan = super.buildMergeOnReadScan();
+          SparkPositionDeltaOperation.this.configuredScan = scan;
+          return scan;
+        }
+      };
+    }
+
+    return lazyScanBuilder;
+  }
+
+  @Override
+  public DeltaWriteBuilder newWriteBuilder(ExtendedLogicalWriteInfo info) {
+    if (lazyWriteBuilder == null) {
+      lazyWriteBuilder = new SparkPositionDeltaWriteBuilder(
+          spark, table, command, configuredScan,

Review comment:
       For merge-on-read, it can be null if the condition evaluates to false and the optimizer replaces the original scan relation with a local relation.

##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);
+  }
+
+  private Schema expectedMetadataSchema() {
+    NestedField partition = MetadataColumns.metadataColumn(table, MetadataColumns.PARTITION_COLUMN_NAME);

Review comment:
       Yeah, exactly.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       Well, it is a little bit tricky. The actual type is defined by the projection. For example, consider MERGE operations. The incoming plan will have wrong nullability for metadata and row ID columns (they will be always nullable as those columns are null for records to insert). However, we never pass row ID or metadata columns with inserts. We only pass them with updates and deletes where those columns have correct values. In other words, the projection has more precise types. The existing logic validates that whatever the projections produce satisfies the target output attributes.
   
   That being said, you are also right that we probably need some validation that we can actually project those columns from `query`...
   
   What do you think, @rdblue?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -61,98 +63,38 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkBatchQueryScan.class);
 
+  private final TableScan scan;
   private final Long snapshotId;

Review comment:
       @rdblue, I reverted back the original change with using `Context` here.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -319,20 +319,45 @@ public static boolean isPromotionAllowed(Type from, Type.PrimitiveType to) {
    */
   public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
                                          Boolean checkNullability, Boolean checkOrdering) {
+    String errMsg = "Cannot write incompatible dataset to table with schema:";
+    checkSchemaCompatibility(errMsg, tableSchema, writeSchema, checkNullability, checkOrdering);
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param context the schema context (e.g. row ID)
+   * @param expectedSchema the expected schema
+   * @param providedSchema the provided schema
+   * @param checkNullability whether to check field nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String context, Schema expectedSchema, Schema providedSchema,
+                                    boolean checkNullability, boolean checkOrdering) {
+    String errMsg = String.format("Provided %s schema is incompatible with expected %s schema:", context, context);
+    checkSchemaCompatibility(errMsg, expectedSchema, providedSchema, checkNullability, checkOrdering);
+  }
+
+  private static void checkSchemaCompatibility(String errMsg, Schema schema, Schema providedSchema,
+                                               boolean checkNullability, boolean checkOrdering) {
     List<String> errors;
     if (checkNullability) {
-      errors = CheckCompatibility.writeCompatibilityErrors(tableSchema, writeSchema, checkOrdering);
+      errors = CheckCompatibility.writeCompatibilityErrors(schema, providedSchema, checkOrdering);
     } else {
-      errors = CheckCompatibility.typeCompatibilityErrors(tableSchema, writeSchema, checkOrdering);
+      errors = CheckCompatibility.typeCompatibilityErrors(schema, providedSchema, checkOrdering);
     }
 
     if (!errors.isEmpty()) {
       StringBuilder sb = new StringBuilder();
-      sb.append("Cannot write incompatible dataset to table with schema:\n")
-          .append(tableSchema)
-          .append("\nwrite schema:")
-          .append(writeSchema)
-          .append("\nProblems:");
+      sb.append(errMsg)
+          .append("\n")
+          .append(schema)
+          .append("\n")
+          .append("provided schema:")
+          .append("\n")
+          .append(providedSchema)
+          .append("\n")
+          .append("problems:");

Review comment:
       Nit: I prefer capitalizing these like they were before. It looks weird to not use sentence case.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       Yeah, I don't quite get why you can't use `query` instead of `originalTable` to look up the row ID attrs and then you'd no longer need `originalTable`. Same with metadata 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.

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       That makes sense, but it sounds to me like we could use `query` and ignore nullability in some cases. I'm more concerned about type widening that is unexpected because we're validating based on what the table produced and not what the query produced.
   
   For cache refreshing, shouldn't we use a callback that captures the table in a closure like we do for other plans?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       I thought about this too but it seems such a sensitive area that gets invoked for every row so I've tried to avoid any extra actions. While try/catch does not cost much unless there is an exception, JVM may not rewrite and perform advanced optimizations on the code inside the block. And having an extra `if` would potentially be even worse.
   
   I checked the code that produces these projections and it seems unlikely we can get an NPE given our tests.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       I thought about this too but it seems such a sensitive area that gets invoked for every row so I've tried to avoid any extra actions. While try/catch does not cost much unless there is an exception, JVM may not rewrite and perform advanced optimizations on the code inside the block.
   
   I double checked the code that produces these projections and it seems unlikely we can get an NPE here given our tests.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask {
+
+  override protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit = {
+    val deltaWriter = writer.asInstanceOf[DeltaWriter[InternalRow]]
+    val operation = row.getInt(0)
+    val rowProjection = projs.rowProjection.orNull
+    val rowIdProjection = projs.rowIdProjection
+    val metadataProjection = projs.metadataProjection.orNull
+
+    operation match {
+      case DELETE_OPERATION =>
+        rowIdProjection.project(row)
+        metadataProjection.project(row)
+        deltaWriter.delete(metadataProjection, rowIdProjection)

Review comment:
       We pass partition and spec id via metadata.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -340,6 +340,42 @@ public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
     }
   }
 
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param ctx the validation context used in the error message
+   * @param expectedSchema the expected schema
+   * @param actualSchema the actual schema
+   * @param checkNullability whether to check nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String ctx, Schema expectedSchema, Schema actualSchema,
+                                    Boolean checkNullability, Boolean checkOrdering) {
+    List<String> errors;
+    if (checkNullability) {
+      errors = CheckCompatibility.writeCompatibilityErrors(expectedSchema, actualSchema, checkOrdering);
+    } else {
+      errors = CheckCompatibility.typeCompatibilityErrors(expectedSchema, actualSchema, checkOrdering);
+    }
+
+    if (!errors.isEmpty()) {
+      StringBuilder sb = new StringBuilder();
+      String header = String.format("Schema is not compatible with expected %s schema:", ctx);
+      sb.append(header)
+          .append("\nexpected schema:")

Review comment:
       Fixed.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -291,17 +247,14 @@ public boolean equals(Object o) {
         readSchema().equals(that.readSchema()) && // compare Spark schemas to ignore field ids
         filterExpressions().toString().equals(that.filterExpressions().toString()) &&
         runtimeFilterExpressions.toString().equals(that.runtimeFilterExpressions.toString()) &&
-        Objects.equals(snapshotId, that.snapshotId) &&
-        Objects.equals(startSnapshotId, that.startSnapshotId) &&
-        Objects.equals(endSnapshotId, that.endSnapshotId) &&
-        Objects.equals(asOfTimestamp, that.asOfTimestamp);
+        ctx.equals(that.ctx);
   }
 
   @Override
   public int hashCode() {
     return Objects.hash(
-        table().name(), readSchema(), filterExpressions().toString(), runtimeFilterExpressions.toString(),
-        snapshotId, startSnapshotId, endSnapshotId, asOfTimestamp);
+        table().name(), readSchema(), filterExpressions().toString(),
+        runtimeFilterExpressions.toString(), ctx);

Review comment:
       Added context to that line.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);

Review comment:
       Fixed.

##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {

Review comment:
       Fixed.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on pull request #3763: Spark: Implement merge-on-read DELETE

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


   This needs another round or two. I am switching to copy-on-write MERGE for now.


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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       Well, it is a little bit tricky. The actual type is defined by the projection. For example, consider MERGE operations. The incoming plan will have wrong nullability for metadata and row ID columns (they will be always nullable as those columns are null for records to insert). However, we never pass row ID or metadata columns with inserts. We only pass them with updates and deletes where those columns have correct values. In other words, the projection has more precise types. The existing logic validates that whatever the projections produce satisfies the target output attributes.
   
   That being said, you are also right that we probably need some validation that we can actually project those columns from query...
   
   What do you think, @rdblue?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)

Review comment:
       Since this is always going to be `DeltaWriter[InternalRow]`, why not cast here instead of in `writeFunc` every time the function is called? That way you can avoid the `instanceof` check for each row.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -157,8 +159,73 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, readConf, schemaWithMetadataColumns(), filterExpressions);
+    Long snapshotId = readConf.snapshotId();
+    Long asOfTimestamp = readConf.asOfTimestamp();
+
+    Preconditions.checkArgument(snapshotId == null || asOfTimestamp == null,
+        "Cannot set both %s and %s to select which table snapshot to scan",
+        SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+
+    Long startSnapshotId = readConf.startSnapshotId();
+    Long endSnapshotId = readConf.endSnapshotId();
+
+    if (snapshotId != null || asOfTimestamp != null) {
+      Preconditions.checkArgument(startSnapshotId == null && endSnapshotId == null,
+          "Cannot set %s and %s for incremental scans when either %s or %s is set",
+          SparkReadOptions.START_SNAPSHOT_ID, SparkReadOptions.END_SNAPSHOT_ID,
+          SparkReadOptions.SNAPSHOT_ID, SparkReadOptions.AS_OF_TIMESTAMP);
+    }
+
+    Preconditions.checkArgument(startSnapshotId != null || endSnapshotId == null,
+        "Cannot set only %s for incremental scans. Please, set %s too.",
+        SparkReadOptions.END_SNAPSHOT_ID, SparkReadOptions.START_SNAPSHOT_ID);
+
+    SparkBatchQueryScan.Context ctx = new SparkBatchQueryScan.Context(
+        snapshotId, asOfTimestamp,
+        startSnapshotId, endSnapshotId);
+
+    return buildBatchQueryScan(ctx);
+  }
+
+  public Scan buildMergeOnReadScan() {
+    Snapshot snapshot = table.currentSnapshot();

Review comment:
       Well, I am debating this. If we want to prohibit setting the snapshot id, then we also have to prohibit setting other time-travel properties (3 more). I was not sure about it so I decided to simply ignore them and create a context based on the current snapshot.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -310,4 +263,62 @@ public String toString() {
         "IcebergScan(table=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)",
         table(), expectedSchema().asStruct(), filterExpressions(), runtimeFilterExpressions, caseSensitive());
   }
+
+  static class Context {

Review comment:
       Like I wrote [here](https://github.com/apache/iceberg/pull/3763#discussion_r771475824), I am not happy with this place either.
   
   For merge-on-read scans, the snapshot ID does not come from `SparkReadConf`. Instead, it comes from the current table snapshot no matter what. `SparkScanBuilder` has more context. Let me think more about this place.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {

Review comment:
       A helper class to avoid passing a huge list of params to methods.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();

Review comment:
       Sounds good.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {

Review comment:
       Same here. Mostly from Spark.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // TODO: validate the row ID and metadata schema

Review comment:
       This seems minor to me. Since we are on a tight schedule, I'd skip it for now.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java
##########
@@ -291,17 +247,14 @@ public boolean equals(Object o) {
         readSchema().equals(that.readSchema()) && // compare Spark schemas to ignore field ids
         filterExpressions().toString().equals(that.filterExpressions().toString()) &&
         runtimeFilterExpressions.toString().equals(that.runtimeFilterExpressions.toString()) &&
-        Objects.equals(snapshotId, that.snapshotId) &&
-        Objects.equals(startSnapshotId, that.startSnapshotId) &&
-        Objects.equals(endSnapshotId, that.endSnapshotId) &&
-        Objects.equals(asOfTimestamp, that.asOfTimestamp);
+        ctx.equals(that.ctx);
   }
 
   @Override
   public int hashCode() {
     return Objects.hash(
-        table().name(), readSchema(), filterExpressions().toString(), runtimeFilterExpressions.toString(),
-        snapshotId, startSnapshotId, endSnapshotId, asOfTimestamp);
+        table().name(), readSchema(), filterExpressions().toString(),
+        runtimeFilterExpressions.toString(), ctx);

Review comment:
       Looks like `runtimeFilterExpressions.toString()` didn't need to move, which would keep the previous line from changing.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // TODO: validate the row ID and metadata schema

Review comment:
       I'd say that this is probably worth doing and isn't going to be a lot of work compared with the rest of this PR.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask = DeltaWithMetadataWritingSparkTask(projections)
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask extends Logging with Serializable {
+
+  protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId)
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask {
+
+  override protected def writeFunc(writer: DataWriter[InternalRow], row: InternalRow): Unit = {
+    val deltaWriter = writer.asInstanceOf[DeltaWriter[InternalRow]]
+    val operation = row.getInt(0)
+    val rowProjection = projs.rowProjection.orNull
+    val rowIdProjection = projs.rowIdProjection
+    val metadataProjection = projs.metadataProjection.orNull
+
+    operation match {
+      case DELETE_OPERATION =>
+        rowIdProjection.project(row)
+        metadataProjection.project(row)
+        deltaWriter.delete(metadataProjection, rowIdProjection)

Review comment:
       We pass partition and spec ID via metadata.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);
+  }
+
+  private Schema expectedMetadataSchema() {
+    NestedField partition = MetadataColumns.metadataColumn(table, MetadataColumns.PARTITION_COLUMN_NAME);

Review comment:
       Ah, now I remember why we need to pass metadata columns back into the source.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import org.apache.iceberg.DistributionMode;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriteBuilder;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.read.Scan;
+import org.apache.spark.sql.types.StructType;
+
+class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final SparkWriteConf writeConf;
+  private final ExtendedLogicalWriteInfo info;
+  private final boolean handleTimestampWithoutZone;
+  private final boolean checkNullability;
+  private final boolean checkOrdering;
+
+  SparkPositionDeltaWriteBuilder(SparkSession spark, Table table, Command command, Scan scan,
+                                 IsolationLevel isolationLevel, ExtendedLogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.command = command;
+    this.scan = (SparkBatchQueryScan) scan;
+    this.isolationLevel = isolationLevel;
+    this.writeConf = new SparkWriteConf(spark, table, info.options());
+    this.info = info;
+    this.handleTimestampWithoutZone = writeConf.handleTimestampWithoutZone();
+    this.checkNullability = writeConf.checkNullability();
+    this.checkOrdering = writeConf.checkOrdering();
+  }
+
+  @Override
+  public DeltaWrite build() {
+    Preconditions.checkArgument(handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    Schema dataSchema = dataSchema();
+    if (dataSchema != null) {
+      TypeUtil.validateWriteSchema(table.schema(), dataSchema, checkNullability, checkOrdering);
+    }
+
+    Schema expectedRowIdSchema = expectedRowIdSchema();
+    Schema rowIdSchema = SparkSchemaUtil.convert(expectedRowIdSchema, info.rowIdSchema());
+    TypeUtil.validateSchema("row ID", expectedRowIdSchema, rowIdSchema, checkNullability, checkOrdering);
+
+    Schema expectedMetadataSchema = expectedMetadataSchema();
+    Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, info.metadataSchema());
+    TypeUtil.validateSchema("metadata", expectedMetadataSchema, metadataSchema, checkNullability, checkOrdering);
+
+    SparkUtil.validatePartitionTransforms(table.spec());
+
+    Distribution distribution = SparkDistributionAndOrderingUtil.buildPositionDeltaDistribution(
+        table, command, distributionMode());
+    SortOrder[] ordering = SparkDistributionAndOrderingUtil.buildPositionDeltaOrdering(
+        table, command, distribution);
+
+    return new SparkPositionDeltaWrite(
+        spark, table, command, scan, isolationLevel, writeConf,
+        info, dataSchema, distribution, ordering);
+  }
+
+  private Schema dataSchema() {
+    StructType dataSparkType = info.schema();
+    return dataSparkType != null ? SparkSchemaUtil.convert(table.schema(), dataSparkType) : null;
+  }
+
+  private Schema expectedRowIdSchema() {
+    return new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION);
+  }
+
+  private Schema expectedMetadataSchema() {

Review comment:
       Nit: this could be static and pass in the table.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {

Review comment:
       Got rid of the state here. Did not make it static as we can't have static methods in non-static inner classes.

##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {

Review comment:
       Added a comment.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {

Review comment:
       The incoming fields are probably fine because they're coming from query via `rowIdProjection`. For the output fields, I think it makes sense to go back to what the table requested. Since the output relation, `table` is probably a `V2Relation` that is wrapping the `RowLevelOperationTable`, we should actually be able to recover the requested fields without using `originalTable.
   
   I think that makes the most sense: we want to validate that the incoming fields (`query` or `rowIdProjection`) satisfy the requirements from the operation. The original table doesn't really need to 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -153,8 +158,90 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {

Review comment:
       @rdblue, I am not entirely happy with this place but it is probably better than using a context.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {

Review comment:
       Why not resolve the attrs against the child `Projection`? That's where the data is coming from. So you'd be finding the row ID fields that are coming from the incoming data that will be extracted by `projections.rowIdProjection`.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def operation: SupportsDelta = {
+    EliminateSubqueryAliases(table) match {
+      case DataSourceV2Relation(RowLevelOperationTable(_, operation), _, _, _, _) =>
+        operation match {
+          case supportsDelta: SupportsDelta =>
+            supportsDelta
+          case _ =>
+            throw new AnalysisException(s"Operation $operation is not a delta operation")
+        }
+      case _ =>
+        throw new AnalysisException(s"Cannot retrieve row-level operation from $table")
+    }
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None =>
+        true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    val rowIdAttrs = ExtendedV2ExpressionUtils.resolveRefs[AttributeReference](
+      operation.rowId,
+      originalTable)
+
+    projections.rowIdProjection.schema.forall { field =>
+      rowIdAttrs.exists(rowIdAttr => isCompatible(field, rowIdAttr))
+    }
+  }
+
+  private def metadataAttrsResolved: Boolean = {

Review comment:
       Same here, @rdblue.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/InternalRowProjection.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class InternalRowProjection(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {

Review comment:
       Since this implements `InternalRow`, you might consider renaming it to `ProjectingInternalRow`.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/SparkDistributionAndOrderingUtil.java
##########
@@ -96,6 +100,44 @@ public static Distribution buildCopyOnWriteDistribution(Table table, Command com
     }
   }
 
+  public static Distribution buildPositionDeltaDistribution(Table table, Command command,
+                                                            DistributionMode distributionMode) {
+    if (command == DELETE) {
+      return positionDeleteDistribution(distributionMode);
+    } else {
+      throw new IllegalArgumentException("Only position deletes are currently supported");
+    }
+  }
+
+  private static Distribution positionDeleteDistribution(DistributionMode distributionMode) {
+    switch (distributionMode) {
+      case NONE:
+        return Distributions.unspecified();
+
+      case HASH:
+        Expression[] clustering = new Expression[]{SPEC_ID, PARTITION};
+        return Distributions.clustered(clustering);
+
+      case RANGE:
+        SortOrder[] ordering = new SortOrder[]{SPEC_ID_ORDER, PARTITION_ORDER, FILE_PATH_ORDER};
+        return Distributions.ordered(ordering);
+
+      default:
+        throw new IllegalArgumentException("Unsupported distribution mode: " + distributionMode);
+    }
+  }
+
+  public static SortOrder[] buildPositionDeltaOrdering(Table table, Command command, Distribution distribution) {
+    // the spec requires position delete files to be sorted by file and pos
+    SortOrder[] deleteOrdering = new SortOrder[]{SPEC_ID_ORDER, PARTITION_ORDER, FILE_PATH_ORDER, ROW_POSITION_ORDER};

Review comment:
       Seems like this should go in the `command == Command.DELETE` block, but it's minor.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec

Review comment:
       It would be nice to have a bit more detail about what's happening here: the projection is from the union partition type from all specs to a specific partition types for the given spec.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {

Review comment:
       Should we move these inner classes to the top level? This file is fairly large.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {
+    private final Schema dataSchema;
+    private final StructType dataSparkType;
+    private final FileFormat dataFileFormat;
+    private final long targetDataFileSize;
+    private final StructType deleteSparkType;
+    private final StructType metadataSparkType;
+    private final FileFormat deleteFileFormat;
+    private final long targetDeleteFileSize;
+    private final boolean fanoutWriterEnabled;
+
+    Context(Schema dataSchema, SparkWriteConf writeConf, ExtendedLogicalWriteInfo info) {

Review comment:
       I'm not sure how much this class actually helps because most of what it does is returning values that are from `info` and `writeConf`. Is the value of this that the `SparkWriteConf` is not `Serializable`? If so, I think it would be good to note that in the class's javadoc.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/ProjectingInternalRow.scala
##########
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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
+
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.catalyst.util.MapData
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.Decimal
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.types.CalendarInterval
+import org.apache.spark.unsafe.types.UTF8String
+
+/**
+ * An InternalRow that projects particular columns from another InternalRow without copying
+ * the underlying data.
+ */
+case class ProjectingInternalRow(schema: StructType, colOrdinals: Seq[Int]) extends InternalRow {
+  assert(schema.size == colOrdinals.size)
+
+  private var row: InternalRow = _
+
+  override def numFields: Int = colOrdinals.size
+
+  def project(row: InternalRow): Unit = {
+    this.row = row
+  }
+
+  override def setNullAt(i: Int): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def update(i: Int, value: Any): Unit = {
+    throw new UnsupportedOperationException("Cannot modify InternalRowProjection")
+  }
+
+  override def copy(): InternalRow = {

Review comment:
       @rdblue, I think the underlying row may not necessarily be `GenericInternalRow`. I think converting `UnsafeRow` values may be more expensive than doing a copy. What do you think?




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelCommand.scala
##########
@@ -54,6 +60,50 @@ trait RewriteRowLevelCommand extends Rule[LogicalPlan] {
     }
   }
 
+  protected def buildWriteDeltaProjections(
+      plan: LogicalPlan,
+      rowAttrs: Seq[Attribute],
+      rowIdAttrs: Seq[Attribute],
+      metadataAttrs: Seq[Attribute]): WriteDeltaProjections = {
+
+    val rowProjection = if (rowAttrs.nonEmpty) {
+      Some(newProjection(plan, rowAttrs, usePlanTypes = true))
+    } else {
+      None
+    }
+
+    // in MERGE, the plan may contain both delete and insert records that may affect
+    // the nullability of metadata columns (e.g. metadata columns for new records are always null)
+    // since metadata columns are never passed with new records to insert,
+    // use the actual metadata column type instead of the one present in the plan
+
+    val rowIdProjection = newProjection(plan, rowIdAttrs, usePlanTypes = false)
+
+    val metadataProjection = if (metadataAttrs.nonEmpty) {
+      Some(newProjection(plan, metadataAttrs, usePlanTypes = false))
+    } else {
+      None
+    }
+
+    WriteDeltaProjections(rowProjection, rowIdProjection, metadataProjection)
+  }
+
+  // the projection is done by name, ignoring expr IDs
+  private def newProjection(

Review comment:
       Renamed.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {

Review comment:
       I followed the same approach we have in `SparkWrite`. This one is not static to share the state. I think it would be possible to split this into multiple files but I'd probably do that together with `SparkWrite`.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java
##########
@@ -0,0 +1,461 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.source;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.IsolationLevel;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.RowDelta;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.SnapshotSummary;
+import org.apache.iceberg.SnapshotUpdate;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.DeleteWriteResult;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.CharSequenceSet;
+import org.apache.iceberg.util.StructProjection;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.connector.iceberg.write.DeltaBatchWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter;
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriterFactory;
+import org.apache.spark.sql.connector.iceberg.write.ExtendedLogicalWriteInfo;
+import org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.iceberg.IsolationLevel.SERIALIZABLE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.MERGE;
+import static org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.UPDATE;
+
+class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final Command command;
+  private final SparkBatchQueryScan scan;
+  private final IsolationLevel isolationLevel;
+  private final Context ctx;
+  private final String applicationId;
+  private final boolean wapEnabled;
+  private final String wapId;
+  private final Map<String, String> extraSnapshotMetadata;
+  private final Distribution requiredDistribution;
+  private final SortOrder[] requiredOrdering;
+
+  SparkPositionDeltaWrite(SparkSession spark, Table table, Command command, SparkBatchQueryScan scan,
+                          IsolationLevel isolationLevel, SparkWriteConf writeConf,
+                          ExtendedLogicalWriteInfo info, Schema dataSchema,
+                          Distribution requiredDistribution, SortOrder[] requiredOrdering) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.command = command;
+    this.scan = scan;
+    this.isolationLevel = isolationLevel;
+    this.ctx = new Context(dataSchema, writeConf, info);
+    this.applicationId = spark.sparkContext().applicationId();
+    this.wapEnabled = writeConf.wapEnabled();
+    this.wapId = writeConf.wapId();
+    this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata();
+    this.requiredDistribution = requiredDistribution;
+    this.requiredOrdering = requiredOrdering;
+  }
+
+  @Override
+  public Distribution requiredDistribution() {
+    return requiredDistribution;
+  }
+
+  @Override
+  public SortOrder[] requiredOrdering() {
+    return requiredOrdering;
+  }
+
+  @Override
+  public DeltaBatchWrite toBatch() {
+    return new PositionDeltaBatchWrite();
+  }
+
+  private static <T extends ContentFile<T>> void cleanFiles(FileIO io, Iterable<T> files) {
+    Tasks.foreach(files)
+        .throwFailureWhenFinished()
+        .noRetry()
+        .run(file -> io.deleteFile(file.path().toString()));
+  }
+
+  private class PositionDeltaBatchWrite implements DeltaBatchWrite {
+
+    @Override
+    public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table));
+      return new PositionDeltaWriteFactory(tableBroadcast, ctx);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      RowDelta rowDelta = table.newRowDelta();
+
+      CharSequenceSet referencedDataFiles = CharSequenceSet.empty();
+
+      int addedDataFilesCount = 0;
+      int addedDeleteFilesCount = 0;
+
+      for (WriterCommitMessage message : messages) {
+        DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+
+        for (DataFile dataFile : taskCommit.dataFiles()) {
+          rowDelta.addRows(dataFile);
+          addedDataFilesCount += 1;
+        }
+
+        for (DeleteFile deleteFile : taskCommit.deleteFiles()) {
+          rowDelta.addDeletes(deleteFile);
+          addedDeleteFilesCount += 1;
+        }
+
+        referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles()));
+      }
+
+      // the scan may be null if the optimizer replaces it with an empty relation (e.g. the cond is false)
+      // no validation is needed in this case as the command does not depend on the scanned table state
+      if (scan != null) {
+        Expression conflictDetectionFilter = conflictDetectionFilter();
+        rowDelta.conflictDetectionFilter(conflictDetectionFilter);
+
+        rowDelta.validateDataFilesExist(referencedDataFiles);
+
+        if (scan.snapshotId() != null) {
+          rowDelta.validateFromSnapshot(scan.snapshotId());
+        }
+
+        if (command == UPDATE || command == MERGE) {
+          rowDelta.validateDeletedFiles();
+          rowDelta.validateNoConflictingDeleteFiles();
+        }
+
+        if (isolationLevel == SERIALIZABLE) {
+          rowDelta.validateNoConflictingDataFiles();
+        }
+
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files " +
+            "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)",
+            addedDataFilesCount, addedDeleteFilesCount, scan.snapshotId(), conflictDetectionFilter, isolationLevel);
+        commitOperation(rowDelta, commitMsg);
+
+      } else {
+        String commitMsg = String.format(
+            "position delta with %d data files and %d delete files (no validation)",
+            addedDataFilesCount, addedDeleteFilesCount);
+        commitOperation(rowDelta, commitMsg);
+      }
+    }
+
+    private Expression conflictDetectionFilter() {
+      Expression filter = Expressions.alwaysTrue();
+
+      for (Expression expr : scan.filterExpressions()) {
+        filter = Expressions.and(filter, expr);
+      }
+
+      return filter;
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeltaTaskCommit taskCommit = (DeltaTaskCommit) message;
+          cleanFiles(table.io(), Arrays.asList(taskCommit.dataFiles()));
+          cleanFiles(table.io(), Arrays.asList(taskCommit.deleteFiles()));
+        }
+      }
+    }
+
+    private void commitOperation(SnapshotUpdate<?> operation, String description) {
+      LOG.info("Committing {} to table {}", description, table);
+      if (applicationId != null) {
+        operation.set("spark.app.id", applicationId);
+      }
+
+      extraSnapshotMetadata.forEach(operation::set);
+
+      if (wapEnabled && wapId != null) {
+        // write-audit-publish is enabled for this table and job
+        // stage the changes without changing the current snapshot
+        operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId);
+        operation.stageOnly();
+      }
+
+      long start = System.currentTimeMillis();
+      operation.commit(); // abort is automatically called if this fails
+      long duration = System.currentTimeMillis() - start;
+      LOG.info("Committed in {} ms", duration);
+    }
+  }
+
+  public static class DeltaTaskCommit implements WriterCommitMessage {
+    private final DataFile[] dataFiles;
+    private final DeleteFile[] deleteFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeltaTaskCommit(DeleteWriteResult result) {
+      this.dataFiles = new DataFile[0];
+      this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]);
+      this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]);
+    }
+
+    DataFile[] dataFiles() {
+      return dataFiles;
+    }
+
+    DeleteFile[] deleteFiles() {
+      return deleteFiles;
+    }
+
+    public CharSequence[] referencedDataFiles() {
+      return referencedDataFiles;
+    }
+  }
+
+  private static class PositionDeltaWriteFactory implements DeltaWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final Context ctx;
+
+    PositionDeltaWriteFactory(Broadcast<Table> tableBroadcast, Context ctx) {
+      this.tableBroadcast = tableBroadcast;
+      this.ctx = ctx;
+    }
+
+    @Override
+    public DeltaWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+      Map<Integer, PartitionSpec> specs = table.specs();
+      FileIO io = table.io();
+
+      OutputFileFactory deleteFileFactory = OutputFileFactory.builderFor(table, partitionId, taskId)
+          .format(ctx.deleteFileFormat())
+          .build();
+
+      SparkFileWriterFactory writerFactory = SparkFileWriterFactory.builderFor(table)
+          .dataFileFormat(ctx.dataFileFormat())
+          .dataSchema(ctx.dataSchema())
+          .dataSparkType(ctx.dataSparkType())
+          .deleteFileFormat(ctx.deleteFileFormat())
+          .positionDeleteSparkType(ctx.deleteSparkType())
+          .build();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+      StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType);
+      InternalRowWrapper partitionRowWrapper = new InternalRowWrapper(sparkPartitionType);
+
+      // build projections that would project needed partitions fields for each spec
+      Map<Integer, StructProjection> partitionProjections = Maps.newHashMap();
+      specs.forEach((specID, spec) ->
+          partitionProjections.put(specID, StructProjection.create(partitionType, spec.partitionType()))
+      );
+
+      if (ctx.dataSchema() == null) {
+        return new DeleteOnlyDeltaWriter(
+            writerFactory, deleteFileFactory, io, ctx,
+            specs, partitionRowWrapper, partitionProjections);
+
+      } else {
+        throw new IllegalArgumentException("Only merge-on-read deletes are currently supported");
+      }
+    }
+  }
+
+  private static class DeleteOnlyDeltaWriter implements DeltaWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> delegate;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final Map<Integer, StructProjection> partitionProjections;
+    private final int specIdOrdinal;
+    private final int partitionOrdinal;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+
+    private boolean closed = false;
+
+    DeleteOnlyDeltaWriter(SparkFileWriterFactory writerFactory,
+                          OutputFileFactory deleteFileFactory, FileIO io,
+                          Context ctx, Map<Integer, PartitionSpec> specs,
+                          InternalRowWrapper partitionRowWrapper,
+                          Map<Integer, StructProjection> partitionProjections) {
+
+      this.delegate = new ClusteredPositionDeleteWriter<>(
+          writerFactory, deleteFileFactory, io,
+          ctx.deleteFileFormat(), ctx.targetDeleteFileSize());
+      this.positionDelete = PositionDelete.create();
+      this.io = io;
+      this.specs = specs;
+      this.partitionRowWrapper = partitionRowWrapper;
+      this.partitionProjections = partitionProjections;
+      this.specIdOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.SPEC_ID.name());
+      this.partitionOrdinal = ctx.metadataSparkType().fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME);
+      this.fileOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.FILE_PATH.name());
+      this.positionOrdinal = ctx.deleteSparkType().fieldIndex(MetadataColumns.ROW_POSITION.name());
+    }
+
+    @Override
+    public void delete(InternalRow meta, InternalRow id) throws IOException {
+      int specId = meta.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = meta.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      StructProjection partitionProjection = partitionProjections.get(specId);
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = id.getString(fileOrdinal);
+      long position = id.getLong(positionOrdinal);
+      positionDelete.set(file, position, null);
+      delegate.write(positionDelete, spec, partitionProjection);
+    }
+
+    @Override
+    public void update(InternalRow metadata, InternalRow id, InternalRow row) {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement update");
+    }
+
+    @Override
+    public void insert(InternalRow row) throws IOException {
+      throw new UnsupportedOperationException(this.getClass().getName() + " does not implement insert");
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      return new DeltaTaskCommit(result);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult result = delegate.result();
+      cleanFiles(io, result.deleteFiles());
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        delegate.close();
+        this.closed = true;
+      }
+    }
+  }
+
+  private static class Context implements Serializable {
+    private final Schema dataSchema;
+    private final StructType dataSparkType;
+    private final FileFormat dataFileFormat;
+    private final long targetDataFileSize;
+    private final StructType deleteSparkType;
+    private final StructType metadataSparkType;
+    private final FileFormat deleteFileFormat;
+    private final long targetDeleteFileSize;
+    private final boolean fanoutWriterEnabled;
+
+    Context(Schema dataSchema, SparkWriteConf writeConf, ExtendedLogicalWriteInfo info) {

Review comment:
       The arguments don't need to be passed in, you can just pass these 3 and set instance fields from 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  // TODO: validate the row ID and metadata schema

Review comment:
       Added some validation.




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

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,46 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {
+    createAndInitTable("id INT, dep STRING, category STRING");
+
+    // write an unpartitioned file
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");
+
+    // write a file partitioned by dep
+    sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName);
+    append(tableName,
+        "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\" }\n" +
+            "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }");
+
+    // write a file partitioned by dep and category
+    sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName);
+    append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}");

Review comment:
       The idea of this test is to create files with `id = 1` in a number of specs and then issue a single delete that would remove all matching records and write delete files that belong to multiple specs. That's why multiple files contain the same record.




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteDelta.scala
##########
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis.NamedRelation
+import org.apache.spark.sql.catalyst.expressions.NamedExpression
+import org.apache.spark.sql.catalyst.util.CharVarcharUtils
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils.OPERATION_COLUMN
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.StructField
+
+/**
+ * Writes a delta of rows to an existing table.
+ */
+case class WriteDelta(
+    table: NamedRelation,
+    query: LogicalPlan,
+    originalTable: NamedRelation,
+    projections: WriteDeltaProjections,
+    write: Option[DeltaWrite] = None) extends V2WriteCommandLike {
+
+  override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write)
+
+  private def operationResolved: Boolean = {
+    val attr = query.output.head
+    attr.name == OPERATION_COLUMN && attr.dataType == IntegerType && !attr.nullable
+  }
+
+  private def rowAttrsResolved: Boolean = {
+    table.skipSchemaResolution || (projections.rowProjection match {
+      case Some(projection) =>
+        table.output.size == projection.schema.size &&
+          projection.schema.zip(table.output).forall { case (field, outAttr) =>
+            isCompatible(field, outAttr)
+          }
+      case None => true
+    })
+  }
+
+  private def rowIdAttrsResolved: Boolean = {
+    projections.rowIdProjection.schema.forall { field =>
+      originalTable.resolve(Seq(field.name), conf.resolver) match {
+        case Some(outAttr) => isCompatible(field, outAttr)
+        case None => false
+      }
+    }
+  }
+
+  private def metadataAttrsResolved: Boolean = {
+    projections.metadataProjection match {
+      case Some(projection) =>
+        projection.schema.forall { field =>
+          originalTable.metadataOutput.exists(metadataAttr => isCompatible(field, metadataAttr))

Review comment:
       Same here. Shouldn't these be looked up in `query` since that's what produces the row that the metadata projection wraps?




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -336,7 +329,47 @@ public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
       for (String error : errors) {
         sb.append("\n* ").append(error);
       }
-      throw new IllegalArgumentException(sb.toString());
+      return sb.toString();
+    });
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param ctx the validation context used in the error message
+   * @param expectedSchema the expected schema
+   * @param actualSchema the actual schema
+   * @param checkNullability whether to check nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String ctx, Schema expectedSchema, Schema actualSchema,
+                                    Boolean checkNullability, Boolean checkOrdering) {
+    validateSchema(expectedSchema, actualSchema, checkNullability, checkOrdering, (errors) -> {
+      StringBuilder sb = new StringBuilder();
+      String header = String.format("Schema is not compatible with expected %s schema:", ctx);
+      sb.append(header)
+          .append(expectedSchema)
+          .append("\nactual schema:")
+          .append(actualSchema)
+          .append("\nproblems:");
+      for (String error : errors) {
+        sb.append("\n* ").append(error);
+      }
+      return sb.toString();
+    });
+  }
+
+  private static void validateSchema(Schema expectedSchema, Schema actualSchema, Boolean checkNullability,

Review comment:
       Yeah, seems like just adding the context part is all you need 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.

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

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



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


[GitHub] [iceberg] aokolnychyi commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java
##########
@@ -749,6 +772,48 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException {
     spark.sql("UNCACHE TABLE tmp");
   }
 
+  @Test
+  public void testDeleteWithMultipleSpecs() {

Review comment:
       @rdblue, I modified this test. Let me know if I got you 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.

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: spark/v3.2/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteDeltaExec.scala
##########
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * 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.SparkEnv
+import org.apache.spark.SparkException
+import org.apache.spark.TaskContext
+import org.apache.spark.internal.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.catalyst.util.WriteDeltaProjections
+import org.apache.spark.sql.connector.iceberg.write.DeltaWrite
+import org.apache.spark.sql.connector.iceberg.write.DeltaWriter
+import org.apache.spark.sql.connector.write.BatchWrite
+import org.apache.spark.sql.connector.write.DataWriter
+import org.apache.spark.sql.connector.write.DataWriterFactory
+import org.apache.spark.sql.connector.write.PhysicalWriteInfoImpl
+import org.apache.spark.sql.connector.write.WriterCommitMessage
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.metric.CustomMetrics
+import org.apache.spark.sql.execution.metric.SQLMetric
+import org.apache.spark.util.LongAccumulator
+import org.apache.spark.util.Utils
+import scala.util.control.NonFatal
+
+/**
+ * Physical plan node to write a delta of rows to an existing table.
+ */
+case class WriteDeltaExec(
+    query: SparkPlan,
+    refreshCache: () => Unit,
+    projections: WriteDeltaProjections,
+    write: DeltaWrite) extends ExtendedV2ExistingTableWriteExec[DeltaWriter[InternalRow]] {
+
+  override lazy val references: AttributeSet = query.outputSet
+  override lazy val stringArgs: Iterator[Any] = Iterator(query, write)
+
+  override lazy val writingTask: WritingSparkTask[DeltaWriter[InternalRow]] = {
+    DeltaWithMetadataWritingSparkTask(projections)
+  }
+
+  override protected def withNewChildInternal(newChild: SparkPlan): WriteDeltaExec = {
+    copy(query = newChild)
+  }
+}
+
+// a trait similar to V2ExistingTableWriteExec but supports custom write tasks
+trait ExtendedV2ExistingTableWriteExec[W <: DataWriter[InternalRow]] extends V2ExistingTableWriteExec {
+  def writingTask: WritingSparkTask[W]
+
+  protected override def writeWithV2(batchWrite: BatchWrite): Seq[InternalRow] = {
+    val rdd: RDD[InternalRow] = {
+      val tempRdd = query.execute()
+      // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single
+      // partition rdd to make sure we at least set up one write task to write the metadata.
+      if (tempRdd.partitions.length == 0) {
+        sparkContext.parallelize(Array.empty[InternalRow], 1)
+      } else {
+        tempRdd
+      }
+    }
+    // introduce a local var to avoid serializing the whole class
+    val task = writingTask
+    val writerFactory = batchWrite.createBatchWriterFactory(
+      PhysicalWriteInfoImpl(rdd.getNumPartitions))
+    val useCommitCoordinator = batchWrite.useCommitCoordinator
+    val messages = new Array[WriterCommitMessage](rdd.partitions.length)
+    val totalNumRowsAccumulator = new LongAccumulator()
+
+    logInfo(s"Start processing data source write support: $batchWrite. " +
+      s"The input RDD has ${messages.length} partitions.")
+
+    // Avoid object not serializable issue.
+    val writeMetrics: Map[String, SQLMetric] = customMetrics
+
+    try {
+      sparkContext.runJob(
+        rdd,
+        (context: TaskContext, iter: Iterator[InternalRow]) =>
+          task.run(writerFactory, context, iter, useCommitCoordinator, writeMetrics),
+        rdd.partitions.indices,
+        (index, result: DataWritingSparkTaskResult) => {
+          val commitMessage = result.writerCommitMessage
+          messages(index) = commitMessage
+          totalNumRowsAccumulator.add(result.numRows)
+          batchWrite.onDataWriterCommit(commitMessage)
+        }
+      )
+
+      logInfo(s"Data source write support $batchWrite is committing.")
+      batchWrite.commit(messages)
+      logInfo(s"Data source write support $batchWrite committed.")
+      commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value))
+    } catch {
+      case cause: Throwable =>
+        logError(s"Data source write support $batchWrite is aborting.")
+        try {
+          batchWrite.abort(messages)
+        } catch {
+          case t: Throwable =>
+            logError(s"Data source write support $batchWrite failed to abort.")
+            cause.addSuppressed(t)
+            throw QueryExecutionErrors.writingJobFailedError(cause)
+        }
+        logError(s"Data source write support $batchWrite aborted.")
+        cause match {
+          // Only wrap non fatal exceptions.
+          case NonFatal(e) => throw QueryExecutionErrors.writingJobAbortedError(e)
+          case _ => throw cause
+        }
+    }
+
+    Nil
+  }
+}
+
+trait WritingSparkTask[W <: DataWriter[InternalRow]] extends Logging with Serializable {
+
+  protected def writeFunc(writer: W, row: InternalRow): Unit
+
+  def run(
+      writerFactory: DataWriterFactory,
+      context: TaskContext,
+      iter: Iterator[InternalRow],
+      useCommitCoordinator: Boolean,
+      customMetrics: Map[String, SQLMetric]): DataWritingSparkTaskResult = {
+    val stageId = context.stageId()
+    val stageAttempt = context.stageAttemptNumber()
+    val partId = context.partitionId()
+    val taskId = context.taskAttemptId()
+    val attemptId = context.attemptNumber()
+    val dataWriter = writerFactory.createWriter(partId, taskId).asInstanceOf[W]
+
+    var count = 0L
+    // write the data and commit this writer.
+    Utils.tryWithSafeFinallyAndFailureCallbacks(block = {
+      while (iter.hasNext) {
+        if (count % CustomMetrics.NUM_ROWS_PER_UPDATE == 0) {
+          CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+        }
+
+        // Count is here.
+        count += 1
+        writeFunc(dataWriter, iter.next())
+      }
+
+      CustomMetrics.updateMetrics(dataWriter.currentMetricsValues, customMetrics)
+
+      val msg = if (useCommitCoordinator) {
+        val coordinator = SparkEnv.get.outputCommitCoordinator
+        val commitAuthorized = coordinator.canCommit(stageId, stageAttempt, partId, attemptId)
+        if (commitAuthorized) {
+          logInfo(s"Commit authorized for partition $partId (task $taskId, attempt $attemptId, " +
+            s"stage $stageId.$stageAttempt)")
+          dataWriter.commit()
+        } else {
+          val commitDeniedException = QueryExecutionErrors.commitDeniedError(
+            partId, taskId, attemptId, stageId, stageAttempt)
+          logInfo(commitDeniedException.getMessage)
+          // throwing CommitDeniedException will trigger the catch block for abort
+          throw commitDeniedException
+        }
+
+      } else {
+        logInfo(s"Writer for partition ${context.partitionId()} is committing.")
+        dataWriter.commit()
+      }
+
+      logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+
+      DataWritingSparkTaskResult(count, msg)
+
+    })(catchBlock = {
+      // If there is an error, abort this writer
+      logError(s"Aborting commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+      dataWriter.abort()
+      logError(s"Aborted commit for partition $partId (task $taskId, attempt $attemptId, " +
+        s"stage $stageId.$stageAttempt)")
+    }, finallyBlock = {
+      dataWriter.close()
+    })
+  }
+}
+
+case class DeltaWithMetadataWritingSparkTask(
+    projs: WriteDeltaProjections) extends WritingSparkTask[DeltaWriter[InternalRow]] {
+
+  private lazy val rowProjection = projs.rowProjection.orNull
+  private lazy val rowIdProjection = projs.rowIdProjection
+  private lazy val metadataProjection = projs.metadataProjection.orNull

Review comment:
       Looks like this may throw NPE when a projection is null but the operation causes it to be accessed? Is there a better way to fail? Maybe check which ones are null and add cases like `case UPDATE_OPERATION if !hasUpdateProjections => throw ...`




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

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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #3763: Spark: Implement merge-on-read DELETE

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



##########
File path: api/src/main/java/org/apache/iceberg/types/TypeUtil.java
##########
@@ -319,20 +319,45 @@ public static boolean isPromotionAllowed(Type from, Type.PrimitiveType to) {
    */
   public static void validateWriteSchema(Schema tableSchema, Schema writeSchema,
                                          Boolean checkNullability, Boolean checkOrdering) {
+    String errMsg = "Cannot write incompatible dataset to table with schema:";
+    checkSchemaCompatibility(errMsg, tableSchema, writeSchema, checkNullability, checkOrdering);
+  }
+
+  /**
+   * Validates whether the provided schema is compatible with the expected schema.
+   *
+   * @param context the schema context (e.g. row ID)
+   * @param expectedSchema the expected schema
+   * @param providedSchema the provided schema
+   * @param checkNullability whether to check field nullability
+   * @param checkOrdering whether to check field ordering
+   */
+  public static void validateSchema(String context, Schema expectedSchema, Schema providedSchema,
+                                    boolean checkNullability, boolean checkOrdering) {
+    String errMsg = String.format("Provided %s schema is incompatible with expected %s schema:", context, context);

Review comment:
       Do we need context twice? I think that "expected schema" is nearly equivalent to "expected row ID schema" if "row ID" was used the first time, "Provided row ID schema ...".




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

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

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



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