You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "szehon-ho (via GitHub)" <gi...@apache.org> on 2023/03/06 20:39:10 UTC

[GitHub] [iceberg] szehon-ho opened a new pull request, #7029: Allow writes for position deletes

szehon-ho opened a new pull request, #7029:
URL: https://github.com/apache/iceberg/pull/7029

   This is the last pre-requisite for implementing RewriteDeleteFiles.
   
   It allows writes to the position_deletes metadata table, on condition of rewritte_file_set_id is set.
   
   Part of this pr, which is simple refactoring, is already split out into:  https://github.com/apache/iceberg/pull/6924


-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134334069


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -761,4 +818,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another

Review Comment:
   Yea, I don't see a single case now where 'row' is actually written intiailly as non-null.  But it's just for completeness as its possible in the 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147892219


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {

Review Comment:
   Actually as PositionDeletesTable.specs() returns the transformed specs (which are just identity on the 'partition' field's members), this check won't catch anything.  We dont have any way to get underlying table currently here, and I am think if the user successfully wrote position deletes the first time, the specs must be validated at some point before?
   
   



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147876300


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {

Review Comment:
   Changed



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1154060497


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete files. Hence, it
+ * assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a {@link SparkPositionDeletesRewrite}.
+   *
+   * @param spark Spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf Spark write config
+   * @param writeInfo Spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeletesWriterFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Writer factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeletesWriterFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeletesWriterFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema = positionDeleteRowSchema();
+      StructType deleteSparkType = deleteSparkType();
+      StructType deleteSparkTypeWithoutRow = deleteSparkTypeWithoutRow();
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataSchema(writeSchema)

Review Comment:
   I am not sure we need to set these `dataXXX` methods since we are not writing any data ([here](https://github.com/apache/iceberg/pull/7029#discussion_r1151403977)).



-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1137739114


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -767,4 +824,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema) {
+      this.writerWithRow =
+          new ClusteredPositionDeleteWriter<>(
+              writerFactoryWithRow, deleteFileFactory, table.io(), targetFileSize);
+      this.writerWithoutRow =
+          new ClusteredPositionDeleteWriter<>(
+              writerFactoryWithoutRow, deleteFileFactory, table.io(), targetFileSize);
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);

Review Comment:
   We know we will write only one spec/partition at a time. What about propagating spec and partition via the rewrite coordinator? Then we can remove all projection code from here and avoid useless work 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134329298


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+
+public class ScanTaskSetManager {
+
+  private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager();
+
+  private final Map<Pair<String, String>, List<ScanTask>> tasksMap = Maps.newConcurrentMap();
+
+  private ScanTaskSetManager() {}
+
+  public static ScanTaskSetManager get() {

Review Comment:
   Yea this class is based on pr #6924, will need to rebase again.  I think it should be fine, as its not actually creating an INSTANCE in get() though?  (that is done in class's static initialization, which is thread safe)



-- 
This is an automated message from the Apache Git Service.
To 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] zhongyujiang commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "zhongyujiang (via GitHub)" <gi...@apache.org>.
zhongyujiang commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127283022


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   I have some concerns about this change, the `Delete Formats` spec says the reason why this column type should be `required` is to make sure the statistics of the deleted row values is accurate. I think the reason to make sure the statistics are accurate is because of the manifest reader will use them to filter delete files: 
   ![image](https://user-images.githubusercontent.com/42907416/223306413-38ea612e-8c76-44af-8ee4-740363d75db7.png)
   So I think if this type is changed to optional, the statistics will become unreliable, which may cause the delete manifest entry to be incorrectly filtered? This is just my understanding of the spec, but I'm not sure.



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1129014586


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   I updated the pr with a fix for this , the idea from chatting with @RussellSpitzer offline.  I made SparkWrite.DeleteWriter now a fan-out that can redirect deletes to two files, one that either has 'row' as required struct, or no 'row' at all.  In most case only one will be chosen.  Thanks for the initial 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1151424302


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);

Review Comment:
   You are right. Up to you if you want to keep it or drop.



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1153631227


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,23 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
 import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends BaseFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {
-    Pair<String, String> id = toID(table, fileSetID);
-    Set<DataFile> result = resultMap.get(id);
-    ValidationException.check(
-        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
-
-    return result;
-  }
-
-  public void clearRewrite(Table table, String fileSetID) {
-    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.remove(id);
-  }
-
-  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   Not sure about this one, isn't it directly inherited from base (so no change)?



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

To unsubscribe, e-mail: 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1153663198


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session

Review Comment:
   done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+

Review Comment:
   removed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.

Review Comment:
   thanks, done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,

Review Comment:
   yes, 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134332339


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+
+public class ScanTaskSetManager {
+
+  private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager();
+
+  private final Map<Pair<String, String>, List<ScanTask>> tasksMap = Maps.newConcurrentMap();
+
+  private ScanTaskSetManager() {}
+
+  public static ScanTaskSetManager get() {
+    return INSTANCE;
+  }
+
+  public void stageTasks(Table table, String setID, List<ScanTask> tasks) {
+    Preconditions.checkArgument(
+        tasks != null && tasks.size() > 0, "Cannot stage null or empty tasks");
+    Pair<String, String> id = toID(table, setID);
+    tasksMap.put(id, tasks);
+  }
+
+  public List<ScanTask> fetchTasks(Table table, String setID) {
+    Pair<String, String> id = toID(table, setID);
+    return tasksMap.get(id);
+  }
+
+  public List<ScanTask> removeTasks(Table table, String setID) {

Review Comment:
   Also part of just-merged #6924 .  This is not used in write directly but later by RewriteDeleteFiles strategies.  (ie, you can see FileScanTaskSetManager::removeTasks, it is used by RewriteDataFiles strategies)



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127012441


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   This was necessary for the writer to allow writing position deletes with a row, but still be ok when there is no row.
   
   Currently, the writer code either uses a schema with required "row" field, or a schema without the "row" field.  The former is actually never used.  So changing to optional should have no impact



-- 
This is an automated message from the Apache Git Service.
To 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] amogh-jahagirdar commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134254487


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+
+public class ScanTaskSetManager {
+
+  private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager();
+
+  private final Map<Pair<String, String>, List<ScanTask>> tasksMap = Maps.newConcurrentMap();
+
+  private ScanTaskSetManager() {}
+
+  public static ScanTaskSetManager get() {
+    return INSTANCE;
+  }
+
+  public void stageTasks(Table table, String setID, List<ScanTask> tasks) {
+    Preconditions.checkArgument(
+        tasks != null && tasks.size() > 0, "Cannot stage null or empty tasks");
+    Pair<String, String> id = toID(table, setID);
+    tasksMap.put(id, tasks);
+  }
+
+  public List<ScanTask> fetchTasks(Table table, String setID) {
+    Pair<String, String> id = toID(table, setID);
+    return tasksMap.get(id);
+  }
+
+  public List<ScanTask> removeTasks(Table table, String setID) {

Review Comment:
   Is `removeTasks` actually needed? 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+
+public class ScanTaskSetManager {
+
+  private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager();
+
+  private final Map<Pair<String, String>, List<ScanTask>> tasksMap = Maps.newConcurrentMap();
+
+  private ScanTaskSetManager() {}
+
+  public static ScanTaskSetManager get() {

Review Comment:
   Should this be `synchronized`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -761,4 +818,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another

Review Comment:
   I guess it should say 'or omit 'row' altogether to **ensure** delete file stats accuracy.'



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -761,4 +818,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another

Review Comment:
   Are there any skewness issues possible with fanning out across non-null and null rows? For example one writer becomes responsible for more work than the other? I'm also just trying to see if there's a need to distinguish the different writers



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134329298


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.ScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+
+public class ScanTaskSetManager {
+
+  private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager();
+
+  private final Map<Pair<String, String>, List<ScanTask>> tasksMap = Maps.newConcurrentMap();
+
+  private ScanTaskSetManager() {}
+
+  public static ScanTaskSetManager get() {

Review Comment:
   Yea this is part of pr #6924, just merged, will need to rebase again.  I think it should be fine, as its not actually creating an INSTANCE in get() though?  (that is done in class's static initialization, which is thread safe)



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147892219


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {

Review Comment:
   Actually as PositionDeletesTable.specs() returns the transformed specs (which are just identity on the 'partition' field's members), this check won't catch anything.  We dont have any way to get underlying table.  I think if the user successfully wrote position deletes the first time, the specs must be validated at some point before?
   
   



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#issuecomment-1497863586

   Merged, thanks @aokolnychyi for detailed review, and @zhongyujiang @amogh-jahagirdar for initial 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1146661698


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   Check; https://github.com/apache/iceberg/blob/master/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java#L353



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   Check: https://github.com/apache/iceberg/blob/master/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java#L353



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147890038


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = null;
+      if (partitionOrdinalOption.isDefined()) {
+        int partitionOrdinal = partitionOrdinalOption.get();
+        partition = record.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      }
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partitionProjection);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partitionProjection);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult resultWithRow = writerWithRow.result();
+      DeleteWriteResult resultWithoutRow = writerWithoutRow.result();
+      SparkCleanupUtil.deleteTaskFiles(
+          io,
+          Lists.newArrayList(
+              Iterables.concat(resultWithRow.deleteFiles(), resultWithoutRow.deleteFiles())));

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1152766064


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);
+    Preconditions.checkArgument(
+        partitions.size() == 1, "All scan tasks of %s are expected to have the same partition");

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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1137737706


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -601,6 +622,42 @@ DataFile[] files() {
     }
   }
 
+  public static class DeleteTaskCommit implements WriterCommitMessage {
+    private final DeleteFile[] taskFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeleteTaskCommit(List<DeleteFile> deleteFiles, List<CharSequence> referencedDataFiles) {
+      this.taskFiles = deleteFiles.toArray(new DeleteFile[0]);
+      this.referencedDataFiles = referencedDataFiles.toArray(new CharSequence[0]);
+    }
+
+    // Reports bytesWritten and recordsWritten to the Spark output metrics.
+    // Can only be called in executor.
+    void reportOutputMetrics() {
+      long bytesWritten = 0L;
+      long recordsWritten = 0L;
+      for (DeleteFile dataFile : taskFiles) {
+        bytesWritten += dataFile.fileSizeInBytes();
+        recordsWritten += dataFile.recordCount();
+      }
+
+      TaskContext taskContext = TaskContext$.MODULE$.get();
+      if (taskContext != null) {
+        OutputMetrics outputMetrics = taskContext.taskMetrics().outputMetrics();
+        outputMetrics.setBytesWritten(bytesWritten);
+        outputMetrics.setRecordsWritten(recordsWritten);
+      }
+    }
+
+    DeleteFile[] files() {
+      return taskFiles;
+    }
+
+    CharSequence[] referencedDataFiles() {

Review Comment:
   Is this being used? I don't this we need this to replace old files with new ones.



-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1137736792


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -160,6 +177,10 @@ BatchWrite asRewrite(String fileSetID) {
     return new RewriteFiles(fileSetID);
   }
 
+  BatchWrite asPositionDeletesRewrite(String fileSetId) {

Review Comment:
   I don't think we share much of the logic in `SparkWriteBuilder` or in `SparkWrite`. What about creating separate hierarchy for that like we have for position deltas?
   
   ```
   SparkPositionDeletesRewriteBuilder implements WriteBuilder ...
   SparkPositionDeletesRewrite implements Write ...
   ```
   
   Otherwise, we complicate this class even more.



-- 
This is an automated message from the Apache Git Service.
To 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 #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#issuecomment-1469372105

   I went through the change. Let me do a detailed review round with fresh eyes tomorrow.


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

To unsubscribe, e-mail: 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1139546119


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -767,4 +824,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema) {
+      this.writerWithRow =
+          new ClusteredPositionDeleteWriter<>(
+              writerFactoryWithRow, deleteFileFactory, table.io(), targetFileSize);
+      this.writerWithoutRow =
+          new ClusteredPositionDeleteWriter<>(
+              writerFactoryWithoutRow, deleteFileFactory, table.io(), targetFileSize);
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);

Review Comment:
   I ended up propagating via the ScanTaskSetManager. 
   
   I could go either way, but doing this way does seem a little bit hacky, versus before the code was more useful in more cases.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -601,6 +622,42 @@ DataFile[] files() {
     }
   }
 
+  public static class DeleteTaskCommit implements WriterCommitMessage {
+    private final DeleteFile[] taskFiles;
+    private final CharSequence[] referencedDataFiles;
+
+    DeleteTaskCommit(List<DeleteFile> deleteFiles, List<CharSequence> referencedDataFiles) {
+      this.taskFiles = deleteFiles.toArray(new DeleteFile[0]);
+      this.referencedDataFiles = referencedDataFiles.toArray(new CharSequence[0]);
+    }
+
+    // Reports bytesWritten and recordsWritten to the Spark output metrics.
+    // Can only be called in executor.
+    void reportOutputMetrics() {
+      long bytesWritten = 0L;
+      long recordsWritten = 0L;
+      for (DeleteFile dataFile : taskFiles) {
+        bytesWritten += dataFile.fileSizeInBytes();
+        recordsWritten += dataFile.recordCount();
+      }
+
+      TaskContext taskContext = TaskContext$.MODULE$.get();
+      if (taskContext != null) {
+        OutputMetrics outputMetrics = taskContext.taskMetrics().outputMetrics();
+        outputMetrics.setBytesWritten(bytesWritten);
+        outputMetrics.setRecordsWritten(recordsWritten);
+      }
+    }
+
+    DeleteFile[] files() {
+      return taskFiles;
+    }
+
+    CharSequence[] referencedDataFiles() {

Review Comment:
   Removed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -160,6 +177,10 @@ BatchWrite asRewrite(String fileSetID) {
     return new RewriteFiles(fileSetID);
   }
 
+  BatchWrite asPositionDeletesRewrite(String fileSetId) {

Review Comment:
   Refactored out



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1157856724


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()

Review Comment:
   I think its a bit harder to read if we define in Constructor, like SparkWriteBuilder, as its a bit detached from this code.  I rewrote to define the vars at beginning of the method.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);

Review Comment:
   Good catch, 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147886663


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();

Review Comment:
   Good catch, done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = null;
+      if (partitionOrdinalOption.isDefined()) {
+        int partitionOrdinal = partitionOrdinalOption.get();
+        partition = record.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      }
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partitionProjection);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partitionProjection);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult resultWithRow = writerWithRow.result();

Review Comment:
   Thanks, 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147886802


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = null;
+      if (partitionOrdinalOption.isDefined()) {
+        int partitionOrdinal = partitionOrdinalOption.get();
+        partition = record.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      }
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partitionProjection);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partitionProjection);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult resultWithRow = writerWithRow.result();

Review Comment:
   Thanks, missed the change here, 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147887906


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema

Review Comment:
   As dicussed offline, I chose to make all columns in PositionDeletesTable real columns, makes everything simpler.



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147886292


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);

Review Comment:
   Note, In the code path, there are already some checks in: SparkStagedScan::taskGroups
   
   but added one here for SparkPositionDeletesRewrite explicitly (now this code is in SparkPositionDeletesRewriteBuilder).



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147883803


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147883549


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127012441


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   This was necessary for the writer to allow writing position deletes with "row", but still be ok when there is null "row".
   
   Currently, the writer code either uses a schema with required "row" field as is here, or a schema without the "row" field (see posPathSchema method just below).  This one with required row field is actually not used, so changing to optional should have no impact.  
   
   This is actually more in line with the position-delete schema in the spec, where "row" is optional.



-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1154058565


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,23 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
 import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends BaseFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {
-    Pair<String, String> id = toID(table, fileSetID);
-    Set<DataFile> result = resultMap.get(id);
-    ValidationException.check(
-        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
-
-    return result;
-  }
-
-  public void clearRewrite(Table table, String fileSetID) {
-    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.remove(id);
-  }
-
-  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   I believe we renamed it to be `fetchSetIds` instead of `fetchSetIDs`, so have to keep the old method 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1153662301


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      SparkCleanupUtil.deleteTaskFiles(io, Lists.newArrayList(allDeleteFiles));
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        if (writerWithRow != null) {
+          writerWithRow.close();
+        }
+        if (writerWithoutRow != null) {
+          writerWithoutRow.close();
+        }
+        this.closed = true;
+      }
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithRow() {
+      if (writerWithRow == null) {
+        writerWithRow =
+            new ClusteredPositionDeleteWriter<>(
+                writerFactoryWithRow, deleteFileFactory, io, targetFileSize);
+      }
+      return writerWithRow;
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithoutRow() {
+      if (writerWithoutRow == null) {
+        writerWithoutRow =

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      SparkCleanupUtil.deleteTaskFiles(io, Lists.newArrayList(allDeleteFiles));
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        if (writerWithRow != null) {
+          writerWithRow.close();
+        }
+        if (writerWithoutRow != null) {
+          writerWithoutRow.close();
+        }
+        this.closed = true;
+      }
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithRow() {
+      if (writerWithRow == null) {
+        writerWithRow =

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();

Review Comment:
   Yep, done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1153662772


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits

Review Comment:
   done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =

Review Comment:
   done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1152765877


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =

Review Comment:
   Did, though its a bit more code now to get the type to make the structLikeSet



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);
+    Preconditions.checkArgument(
+        partitions.size() == 1, "All scan tasks of %s are expected to have the same partition");
+    int specId = scanTasks.get(0).spec().specId();

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);

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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1151071080


##########
core/src/main/java/org/apache/iceberg/PartitionsTable.java:
##########
@@ -97,6 +97,7 @@ private static StaticDataTask.Row convertPartition(Partition partition) {
 
   private static Iterable<Partition> partitions(Table table, StaticTableScan scan) {
     CloseableIterable<FileScanTask> tasks = planFiles(scan);
+

Review Comment:
   Unnecessary change?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();

Review Comment:
   What about `taskSetManager` and `tasks` instead of `scanTaskSetManager` and `scanTasks` for shorter vars? Some of the lines below would fit on one line.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);

Review Comment:
   nit: Shall this error message start with a capital letter like others?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();

Review Comment:
   Shall we define this var at the beginning of the method as it is not the first time we refer to it?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetId the id used to identify the source set of files being rewritten
+   * @param newFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetId, Set<F> newFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetId,
+        newFiles.size());
+    Pair<String, String> id = toId(table, fileSetId);
+    resultMap.put(id, newFiles);
+  }
+
+  public Set<F> fetchNewFiles(Table table, String fileSetID) {

Review Comment:
   nit: `fileSetID` -> `fileSetId`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,

Review Comment:
   nit: Redundant second `delete`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);
+    Preconditions.checkArgument(
+        partitions.size() == 1, "All scan tasks of %s are expected to have the same partition");
+    int specId = scanTasks.get(0).spec().specId();

Review Comment:
   What about having helper methods to simplify `build`?
   
   ```
   // all files of rewrite group have same and partition and spec id
   ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
   List<PositionDeletesScanTask> tasks = taskSetManager.fetchTasks(table, fileSetId);
   Preconditions.checkNotNull(tasks, "No tasks found for %s", fileSetId);
   
   PartitionSpec spec = spec(fileSetId, tasks);
   StructLike partition = partition(fileSetId, spec, tasks);
   
   return ...
   ```
   



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(

Review Comment:
   I don't think we write a delta here. What about `PositionDeletesWriterFactory` or something?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits

Review Comment:
   nit: `an required` -> `a required`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id

Review Comment:
   nit: Redundant `and` before `partition and spec id`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =

Review Comment:
   What about 3 extra methods to simplify this block?
   
   ```
   private Schema positionDeleteRowSchema() {
     return new Schema(...);
   }
   
   private StructType deleteSparkType() {
     return new StructType(...);
   }
   
   private StructType deleteSparkTypeWithoutRow() {
     return new StructType(...);
   }
   ```
   
   We could directly call them when constructing `SparkFileWriterFactory`.
   
   ```
   SparkFileWriterFactory writerFactoryWithRow =
       SparkFileWriterFactory.builderFor(table)
           .deleteFileFormat(format)
           .positionDeleteRowSchema(positionDeleteRowSchema())
           .positionDeleteSparkType(deleteSparkType())
           .build();
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      SparkCleanupUtil.deleteTaskFiles(io, Lists.newArrayList(allDeleteFiles));
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        if (writerWithRow != null) {
+          writerWithRow.close();
+        }
+        if (writerWithoutRow != null) {
+          writerWithoutRow.close();
+        }
+        this.closed = true;
+      }
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithRow() {
+      if (writerWithRow == null) {
+        writerWithRow =

Review Comment:
   nit: We usually use `this.varName` whenever setting (only setting, not getting) a field.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);

Review Comment:
   Shall we include all spec IDs we got in the error message?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =

Review Comment:
   Shall we use `StructLikeSet` for partitions?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);
+
+    Set<Integer> specIds =
+        scanTasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Set<StructLike> partitions =
+        scanTasks.stream().map(t -> t.file().partition()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1, "All scan tasks of %s are expected to have same spec id", fileSetId);
+    Preconditions.checkArgument(
+        partitions.size() == 1, "All scan tasks of %s are expected to have the same partition");

Review Comment:
   We define `%s` but don't include a value for `fileSetId`.
   Also, what about including the set of partitions we got?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   That place is different as it writes a delta (delete + insert). I think you should be able to drop `dataXXX` methods and it should work fine. The writer factory will use the default format for data but we are not using that anyway. We just write deletes.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,23 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
 import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends BaseFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {
-    Pair<String, String> id = toID(table, fileSetID);
-    Set<DataFile> result = resultMap.get(id);
-    ValidationException.check(
-        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
-
-    return result;
-  }
-
-  public void clearRewrite(Table table, String fileSetID) {
-    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.remove(id);
-  }
-
-  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   We should keep and deprecate this one too.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.

Review Comment:
   I think there is a typo in the name. Shall we also use `@link` like you do in other places?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session

Review Comment:
   nit: `spark` -> `Spark` in all comments here?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {

Review Comment:
   You are probably right. It should be OK as we don't produce a partition tuple.
   
   What about validation of timestamps? Will we have issues if the persisted row has a timestamp without a timezone? Our `SparkWriteBuilder` has the following call.
   
   ```
   Preconditions.checkArgument(
       handleTimestampWithoutZone || !SparkUtil.hasTimestampWithoutZone(table.schema()),
       SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+

Review Comment:
   Is this empty line intentional?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter

Review Comment:
   nit: `@link` and dot at the end?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();

Review Comment:
   Looks like we need this in two places, what about a common method `allDeleteFiles()`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,416 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete delete files. Hence,
+ * it assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteSparkType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteSparkType)
+              .build();
+
+      StructType deleteSparkTypeWithoutRow =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+              });
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteSparkTypeWithoutRow)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId,
+          partition);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final PartitionSpec spec;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+    private final StructLike partition;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All incoming partition deletes
+     *     are required to have the same spec id.
+     * @param partition partition value of incoming position delete. All incoming partition deletes
+     *     are required to have the same partition.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.spec = table.specs().get(specId);
+      this.partition = partition;
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partition);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partition);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      SparkCleanupUtil.deleteTaskFiles(io, Lists.newArrayList(allDeleteFiles));
+    }
+
+    @Override
+    public void close() throws IOException {
+      if (!closed) {
+        if (writerWithRow != null) {
+          writerWithRow.close();
+        }
+        if (writerWithoutRow != null) {
+          writerWithoutRow.close();
+        }
+        this.closed = true;
+      }
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithRow() {
+      if (writerWithRow == null) {
+        writerWithRow =
+            new ClusteredPositionDeleteWriter<>(
+                writerFactoryWithRow, deleteFileFactory, io, targetFileSize);
+      }
+      return writerWithRow;
+    }
+
+    private ClusteredPositionDeleteWriter<InternalRow> lazyWriterWithoutRow() {
+      if (writerWithoutRow == null) {
+        writerWithoutRow =

Review Comment:
   nit: `this.writerWithoutRow = ...`



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127096509


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   Update: Looks like a few GenericWriter depend on old schema to throw exception if null rows passed in.  This is a change of behavior , but backward compatible.



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127096509


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   Update: Looks like a few GenericWriter depend on this, to throw exception if null rows passed in.  This will thus be a change of behavior , but backward compatible.



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1152765598


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1157906102


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,23 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
 import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends BaseFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {
-    Pair<String, String> id = toID(table, fileSetID);
-    Set<DataFile> result = resultMap.get(id);
-    ValidationException.check(
-        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
-
-    return result;
-  }
-
-  public void clearRewrite(Table table, String fileSetID) {
-    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.remove(id);
-  }
-
-  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   Ah i see, yea, added it back.



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

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] szehon-ho merged pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho merged PR #7029:
URL: https://github.com/apache/iceberg/pull/7029


-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1146660914


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   The name is unfortunate but I think it needs the right format to configure the right Writer.  If not, it seems to pick the default format.



-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1143949281


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {

Review Comment:
   nit: We rarely use `Abstract` in the name. Frequently, we call it `BaseXXX`. 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);

Review Comment:
   Is having `from RewriteCoordinator` in the message helpful? Won't that be clear from the logger class?
   I know we copied this but makes sense to revisit if we change it anyway.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {

Review Comment:
   The name should be generic like `fetchNewFiles`. We can still keep `fetchNewDataFiles` in the implementation for data files, just deprecate it.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();

Review Comment:
   We must use delete file configs.
   
   ```
   this.format = writeConf.deleteFileFormat();
   this.targetFileSize = writeConf.targetDeleteFileSize();
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {

Review Comment:
   nit: `fileSetID` -> `fileSetId`, `newDataFiles` -> `newFiles`.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema

Review Comment:
   I am not sure how this works. I assumed `writeSchema` wouldn't have correct IDs for metadata columns.
   Let me double check this.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {
+    return resultMap.keySet().stream()
+        .filter(e -> e.first().equals(tableUUID(table)))
+        .map(Pair::second)
+        .collect(Collectors.toSet());
+  }
+
+  private Pair<String, String> toID(Table table, String setID) {

Review Comment:
   nit: `toID` -> `toId` like we did in `ScanTaskSetManager` fixing my original mistake.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   nit: `fetchSetIDs` -> `fetchSetIds`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {

Review Comment:
   nit: `fileSetId`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java:
##########
@@ -284,7 +285,11 @@ public WriteBuilder newWriteBuilder(LogicalWriteInfo info) {
     Preconditions.checkArgument(
         snapshotId == null, "Cannot write to table at a specific snapshot: %s", snapshotId);
 
-    return new SparkWriteBuilder(sparkSession(), icebergTable, branch, info);
+    if (icebergTable instanceof PositionDeletesTable) {

Review Comment:
   Seems reasonable. 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = null;
+      if (partitionOrdinalOption.isDefined()) {
+        int partitionOrdinal = partitionOrdinalOption.get();
+        partition = record.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      }
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partitionProjection);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partitionProjection);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult resultWithRow = writerWithRow.result();
+      DeleteWriteResult resultWithoutRow = writerWithoutRow.result();
+      SparkCleanupUtil.deleteTaskFiles(
+          io,
+          Lists.newArrayList(
+              Iterables.concat(resultWithRow.deleteFiles(), resultWithoutRow.deleteFiles())));

Review Comment:
   Can we do what you did in `commit` with an explicit list and if/else blocks?
   
   ```
   List<DeleteFile> allDeleteFiles = Lists.newArrayList();
   ...
   SparkCleanupUtil.deleteTaskFiles(io, allDeleteFiles);
   ```
   
   This formatting is harder to read.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,15 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
-import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends AbstractFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
-
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {

Review Comment:
   We may need to keep this one for compatibility if we rename the parent method.
   Just deprecate and remove in the future.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {

Review Comment:
   Doest it have to be public?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);

Review Comment:
   Shall we add some validation that `scanTasks` is not null?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}

Review Comment:
   nit: Missing `.` at the end?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);

Review Comment:
   Is this being used?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;

Review Comment:
   All files not only have the same spec but also have the same partition. Shall we compute the partition here? We can then drop the logic to derive the partition tuple for every row as it must be same. For unpartitioned tables the partition will be null.
   
   In the future, we may optimize the write and not even pass spec ID and partition.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {
+    return resultMap.keySet().stream()
+        .filter(e -> e.first().equals(tableUUID(table)))
+        .map(Pair::second)
+        .collect(Collectors.toSet());
+  }
+
+  private Pair<String, String> toID(Table table, String setID) {

Review Comment:
   nit: `setID` -> `setId`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+

Review Comment:
   I like this empty line to separate the block below.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   Is it required to configure `dataXXX` methods?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);
+      PartitionSpec spec = specs.get(specId);
+
+      InternalRow partition = null;
+      if (partitionOrdinalOption.isDefined()) {
+        int partitionOrdinal = partitionOrdinalOption.get();
+        partition = record.getStruct(partitionOrdinal, partitionRowWrapper.size());
+      }
+      partitionProjection.wrap(partitionRowWrapper.wrap(partition));
+
+      String file = record.getString(fileOrdinal);
+      long position = record.getLong(positionOrdinal);
+      InternalRow row = record.getStruct(rowOrdinal, rowSize);
+      if (row != null) {
+        positionDelete.set(file, position, row);
+        lazyWriterWithRow().write(positionDelete, spec, partitionProjection);
+      } else {
+        positionDelete.set(file, position, null);
+        lazyWriterWithoutRow().write(positionDelete, spec, partitionProjection);
+      }
+    }
+
+    @Override
+    public WriterCommitMessage commit() throws IOException {
+      close();
+
+      List<DeleteFile> allDeleteFiles = Lists.newArrayList();
+      if (writerWithRow != null) {
+        allDeleteFiles.addAll(writerWithRow.result().deleteFiles());
+      }
+      if (writerWithoutRow != null) {
+        allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles());
+      }
+      return new DeleteTaskCommit(allDeleteFiles);
+    }
+
+    @Override
+    public void abort() throws IOException {
+      close();
+
+      DeleteWriteResult resultWithRow = writerWithRow.result();

Review Comment:
   Writers can be null.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =

Review Comment:
   nit: `deleteSparkType`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {

Review Comment:
   Looks correct.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)

Review Comment:
   Same question.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)

Review Comment:
   We need to double check how this works with row and without row.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema,
+          specId);
+    }
+  }
+
+  /**
+   * Writer for position deletes metadata table.
+   *
+   * <p>Iceberg specifies delete files schema as having either 'row' as an required field, or omits
+   * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence,
+   * this writer, if receiving source position deletes with null and non-null rows, redirects rows
+   * with null 'row' to one file writer, and non-null 'row' to another file writer.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition.
+   */
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final SparkFileWriterFactory writerFactoryWithRow;
+    private final SparkFileWriterFactory writerFactoryWithoutRow;
+    private final OutputFileFactory deleteFileFactory;
+    private final long targetFileSize;
+    private final PositionDelete<InternalRow> positionDelete;
+    private final FileIO io;
+    private final Map<Integer, PartitionSpec> specs;
+    private final InternalRowWrapper partitionRowWrapper;
+    private final StructProjection partitionProjection;
+    private final int specIdOrdinal;
+    private final Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    private boolean closed = false;
+
+    /**
+     * Constructs a DeleteWriter
+     *
+     * @param table position deletes metadata table
+     * @param writerFactoryWithRow writer factory for deletes with non-null 'row'
+     * @param writerFactoryWithoutRow writer factory for deletes with null 'row'
+     * @param deleteFileFactory delete file factory
+     * @param targetFileSize target file size
+     * @param dsSchema schema of incoming dataset of position deletes
+     * @param specId partition spec id of incoming position deletes. All files of this row-group are
+     *     required to have one spec id.
+     */
+    DeleteWriter(
+        Table table,
+        SparkFileWriterFactory writerFactoryWithRow,
+        SparkFileWriterFactory writerFactoryWithoutRow,
+        OutputFileFactory deleteFileFactory,
+        long targetFileSize,
+        StructType dsSchema,
+        int specId) {
+      this.deleteFileFactory = deleteFileFactory;
+      this.targetFileSize = targetFileSize;
+      this.writerFactoryWithRow = writerFactoryWithRow;
+      this.writerFactoryWithoutRow = writerFactoryWithoutRow;
+      this.positionDelete = PositionDelete.create();
+      this.io = table.io();
+      this.specs = table.specs();
+
+      Types.StructType partitionType = Partitioning.partitionType(table);
+
+      this.specIdOrdinal = dsSchema.fieldIndex(PositionDeletesTable.SPEC_ID);
+      this.partitionOrdinalOption =
+          dsSchema.getFieldIndex(PositionDeletesTable.PARTITION).map(a -> (Integer) a);
+      this.partitionRowWrapper = initPartitionRowWrapper(partitionType);
+      this.partitionProjection =
+          StructProjection.create(partitionType, table.specs().get(specId).partitionType());
+
+      this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name());
+      this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name());
+
+      this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME);
+      DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType();
+      Preconditions.checkArgument(
+          type instanceof StructType, "Expected row as struct type but was %s", type);
+      this.rowSize = ((StructType) type).size();
+    }
+
+    @Override
+    public void write(InternalRow record) throws IOException {
+      int specId = record.getInt(specIdOrdinal);

Review Comment:
   Let's compute the spec in the constructor. No need to repeat this for every row.
   
   This method can be as simple as this (where `spec` and `partition` are known in the constructor):
   
   ```
   @Override
   public void write(InternalRow record) throws IOException {
     String file = record.getString(fileOrdinal);
     long position = record.getLong(positionOrdinal);
     InternalRow row = record.getStruct(rowOrdinal, rowSize);
     if (row != null) {
       positionDelete.set(file, position, row);
       lazyWriterWithRow().write(positionDelete, spec, partition);
     } else {
       positionDelete.set(file, position, null);
       lazyWriterWithoutRow().write(positionDelete, spec, partition);
     }
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {

Review Comment:
   I think we need to validate the spec has only supported transforms, just we do in normal writes. This may mean you will have to move the code that computes spec and partition here and pass it to the rewrite rather than computing it in the rewrite. You can also do extra validation that all scan tasks have the same spec and partition.
   
   Should we also validate handling of timestamps, just like in normal writes?



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147883277


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {
+    return resultMap.keySet().stream()
+        .filter(e -> e.first().equals(tableUUID(table)))
+        .map(Pair::second)
+        .collect(Collectors.toSet());
+  }
+
+  private Pair<String, String> toID(Table table, String setID) {

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147884038


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,15 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
-import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
-import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends AbstractFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
-
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {

Review Comment:
   Re-added and deprecated



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {

Review Comment:
   Made package private



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

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1153664561


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetId the id used to identify the source set of files being rewritten
+   * @param newFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetId, Set<F> newFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetId,
+        newFiles.size());
+    Pair<String, String> id = toId(table, fileSetId);
+    resultMap.put(id, newFiles);
+  }
+
+  public Set<F> fetchNewFiles(Table table, String fileSetID) {

Review Comment:
   done



##########
core/src/main/java/org/apache/iceberg/PartitionsTable.java:
##########
@@ -97,6 +97,7 @@ private static StaticDataTask.Row convertPartition(Partition partition) {
 
   private static Iterable<Partition> partitions(Table table, StaticTableScan scan) {
     CloseableIterable<FileScanTask> tasks = planFiles(scan);
+

Review Comment:
   removed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {

Review Comment:
   added



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

To unsubscribe, e-mail: 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1152765334


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(scanTasks, "no scan tasks found for %s", fileSetId);

Review Comment:
   Fixed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.source;
+
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    Preconditions.checkArgument(
+        writeConf.rewrittenFileSetId() != null,
+        "position_deletes table can only be written by RewriteDeleteFiles");
+
+    // all files of rewrite group have same and partition and spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    String fileSetId = writeConf.rewrittenFileSetId();

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1127449707


##########
core/src/main/java/org/apache/iceberg/io/DeleteSchemaUtil.java:
##########
@@ -29,7 +29,7 @@ private static Schema pathPosSchema(Schema rowSchema) {
     return new Schema(
         MetadataColumns.DELETE_FILE_PATH,
         MetadataColumns.DELETE_FILE_POS,
-        Types.NestedField.required(
+        Types.NestedField.optional(

Review Comment:
   Yea you are right, this is tricky.  It says this in spec:
   
   > 
   
   2147483544 row | required struct<...> [1] | Deleted row values. Omit the column when not storing deleted rows.
   -- | -- | --
   
   > When present in the delete file, row is required because all delete entries must include the row values.
   
   So either, entire position delete file has 'row', or entire file does not have 'row'.  (Currently it seems Spark does not set 'row' at all, ref:  https://github.com/apache/iceberg/blob/master/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java#L436)
   
   I somehow need a way, when compacting delete files, to know whether the original position file all have rows or not.  I am not sure at the moment how to get 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] amogh-jahagirdar commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "amogh-jahagirdar (via GitHub)" <gi...@apache.org>.
amogh-jahagirdar commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134278293


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -761,4 +818,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteRowSchema(positionDeleteRowSchema)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      SparkFileWriterFactory writerFactoryWithoutRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataFileFormat(format)
+              .dataSchema(writeSchema)
+              .dataSparkType(dsSchema)
+              .deleteFileFormat(format)
+              .positionDeleteSparkType(deleteFileType)
+              .build();
+
+      return new DeleteWriter(
+          table,
+          writerFactoryWithRow,
+          writerFactoryWithoutRow,
+          deleteFileFactory,
+          targetFileSize,
+          dsSchema);
+    }
+  }
+
+  private static class DeleteWriter implements DataWriter<InternalRow> {
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithRow;
+    private final ClusteredPositionDeleteWriter<InternalRow> writerWithoutRow;
+    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 Option<Integer> partitionOrdinalOption;
+    private final int fileOrdinal;
+    private final int positionOrdinal;
+    private final int rowOrdinal;
+    private final int rowSize;
+
+    private boolean closed = false;
+
+    /**
+     * Writer for position deletes metadata table.
+     *
+     * <p>Delete files need to either have 'row' as required field, or omit 'row' altogether, for
+     * delete file stats accuracy Hence, this is a fanout writer, redirecting rows with null 'row'
+     * to one delegate, and non-null 'row' to another

Review Comment:
   sorry completely missed this thread https://github.com/apache/iceberg/pull/7029#discussion_r1127012441, which seems to discuss this point



-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1137739594


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java:
##########
@@ -166,7 +170,12 @@ public Write build() {
 
       @Override
       public BatchWrite toBatch() {
-        if (rewrittenFileSetId != null) {
+        if (table instanceof PositionDeletesTable) {

Review Comment:
   Yeah, I'd probably have a separate path for 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] szehon-ho commented on pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#issuecomment-1472989189

   Made suggested changes (refactored to new classe SparkPositionDeletesRewrite from SparkWrite).  
   
   Notes, new classes drop some unused codes from the previous path, like reportMetrics method and cleanupOnAbort flag to control abort behavior.  I assume we can go back to this when we implement the commit manager part, as of now its not clear whether we need this or not.


-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1154048543


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()
+            || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    // all files of rewrite group have same partition and spec id
+    ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> tasks = taskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(tasks, "No scan tasks found for %s", fileSetId);
+    Preconditions.checkArgument(tasks.size() > 0, "No scan tasks found for %s", fileSetId);
+
+    int specId = specId(fileSetId, tasks);
+    StructLike partition = partition(fileSetId, tasks);
+
+    return new SparkPositionDeletesRewrite(
+        spark, table, writeConf, writeInfo, writeSchema, dsSchema, specId, partition);
+  }
+
+  private int specId(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    Set<Integer> specIds = tasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1,
+        "All scan tasks of %s are expected to have same spec id, but got %s",
+        fileSetId,
+        Joiner.on(",").join(specIds));
+    return tasks.get(0).spec().specId();
+  }
+
+  private StructLike partition(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    StructLikeSet partitions = StructLikeSet.create(tasks.get(0).spec().partitionType());
+    partitions.addAll(tasks.stream().map(ContentScanTask::partition).collect(Collectors.toList()));
+    Preconditions.checkArgument(
+        partitions.size() == 1,
+        "All scan tasks of %s are expected to have the same partition",

Review Comment:
   Did we miss `, but got %s` at the end to include partitions?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class BaseFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);

Review Comment:
   I think we are using a wrong class for logging. It should be `BaseFileRewriteCoordinator`.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()

Review Comment:
   I think this part would be easier to read if we define `fileSetId` and `handleTimestampWithoutZone` instance variables, similar to what we have in `SparkWriteBuilder`.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()
+            || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    // all files of rewrite group have same partition and spec id
+    ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> tasks = taskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(tasks, "No scan tasks found for %s", fileSetId);
+    Preconditions.checkArgument(tasks.size() > 0, "No scan tasks found for %s", fileSetId);
+
+    int specId = specId(fileSetId, tasks);
+    StructLike partition = partition(fileSetId, tasks);
+
+    return new SparkPositionDeletesRewrite(
+        spark, table, writeConf, writeInfo, writeSchema, dsSchema, specId, partition);
+  }
+
+  private int specId(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    Set<Integer> specIds = tasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1,
+        "All scan tasks of %s are expected to have same spec id, but got %s",
+        fileSetId,
+        Joiner.on(",").join(specIds));
+    return tasks.get(0).spec().specId();
+  }
+
+  private StructLike partition(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    StructLikeSet partitions = StructLikeSet.create(tasks.get(0).spec().partitionType());
+    partitions.addAll(tasks.stream().map(ContentScanTask::partition).collect(Collectors.toList()));

Review Comment:
   nit: I think you can use `forEach` instead of a temp list.
   
   ```
   tasks.stream().map(ContentScanTask::partition).forEach(partitions::add);
   ```
   
   In any case, I like what you did here. 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete files. Hence, it
+ * assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a {@link SparkPositionDeletesRewrite}.
+   *
+   * @param spark Spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf Spark write config
+   * @param writeInfo Spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeletesWriterFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Writer factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeletesWriterFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeletesWriterFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema = positionDeleteRowSchema();
+      StructType deleteSparkType = deleteSparkType();
+      StructType deleteSparkTypeWithoutRow = deleteSparkTypeWithoutRow();
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataSchema(writeSchema)

Review Comment:
   I am not sure these `dataXXX` methods since we are not writing any data ([here](https://github.com/apache/iceberg/pull/7029#discussion_r1151403977)).



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -18,79 +18,23 @@
  */
 package org.apache.iceberg.spark;
 
-import java.util.Map;
 import java.util.Set;
-import java.util.stream.Collectors;
 import org.apache.iceberg.DataFile;
-import org.apache.iceberg.HasTableOperations;
 import org.apache.iceberg.Table;
-import org.apache.iceberg.TableOperations;
-import org.apache.iceberg.exceptions.ValidationException;
-import org.apache.iceberg.relocated.com.google.common.collect.Maps;
-import org.apache.iceberg.util.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-public class FileRewriteCoordinator {
+public class FileRewriteCoordinator extends BaseFileRewriteCoordinator<DataFile> {
 
-  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
   private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator();
 
-  private final Map<Pair<String, String>, Set<DataFile>> resultMap = Maps.newConcurrentMap();
-
   private FileRewriteCoordinator() {}
 
   public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /**
-   * Called to persist the output of a rewrite action for a specific group. Since the write is done
-   * via a Spark Datasource, we have to propagate the result through this side-effect call.
-   *
-   * @param table table where the rewrite is occurring
-   * @param fileSetID the id used to identify the source set of files being rewritten
-   * @param newDataFiles the new files which have been written
-   */
-  public void stageRewrite(Table table, String fileSetID, Set<DataFile> newDataFiles) {
-    LOG.debug(
-        "Staging the output for {} - fileset {} with {} files",
-        table.name(),
-        fileSetID,
-        newDataFiles.size());
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.put(id, newDataFiles);
-  }
-
-  public Set<DataFile> fetchNewDataFiles(Table table, String fileSetID) {
-    Pair<String, String> id = toID(table, fileSetID);
-    Set<DataFile> result = resultMap.get(id);
-    ValidationException.check(
-        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
-
-    return result;
-  }
-
-  public void clearRewrite(Table table, String fileSetID) {
-    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
-    Pair<String, String> id = toID(table, fileSetID);
-    resultMap.remove(id);
-  }
-
-  public Set<String> fetchSetIDs(Table table) {

Review Comment:
   I believe we renamed it to be `fetchSetIds` instead of `fetchSetIDs`.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");

Review Comment:
   I don't think there is `RewriteDeleteFiles`.
   What about a more generic message like `Can only write to %s via actions", table.name()`?



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

To unsubscribe, e-mail: 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1157856320


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()
+            || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    // all files of rewrite group have same partition and spec id
+    ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> tasks = taskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(tasks, "No scan tasks found for %s", fileSetId);
+    Preconditions.checkArgument(tasks.size() > 0, "No scan tasks found for %s", fileSetId);
+
+    int specId = specId(fileSetId, tasks);
+    StructLike partition = partition(fileSetId, tasks);
+
+    return new SparkPositionDeletesRewrite(
+        spark, table, writeConf, writeInfo, writeSchema, dsSchema, specId, partition);
+  }
+
+  private int specId(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    Set<Integer> specIds = tasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1,
+        "All scan tasks of %s are expected to have same spec id, but got %s",
+        fileSetId,
+        Joiner.on(",").join(specIds));
+    return tasks.get(0).spec().specId();
+  }
+
+  private StructLike partition(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    StructLikeSet partitions = StructLikeSet.create(tasks.get(0).spec().partitionType());
+    partitions.addAll(tasks.stream().map(ContentScanTask::partition).collect(Collectors.toList()));

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+import org.apache.iceberg.io.ClusteredPositionDeleteWriter;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.OutputFileFactory;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkWriteConf;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite position delete files. Hence, it
+ * assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all
+ * have the same partition spec id and partition values.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+  private final StructLike partition;
+
+  /**
+   * Constructs a {@link SparkPositionDeletesRewrite}.
+   *
+   * @param spark Spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf Spark write config
+   * @param writeInfo Spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   * @param specId spec id of position deletes
+   * @param partition partition value of position deletes
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema,
+      int specId,
+      StructLike partition) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.deleteFileFormat();
+    this.targetFileSize = writeConf.targetDeleteFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+    this.specId = specId;
+    this.partition = partition;
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeletesWriterFactory(
+          tableBroadcast,
+          queryId,
+          format,
+          targetFileSize,
+          writeSchema,
+          dsSchema,
+          specId,
+          partition);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Writer factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeletesWriterFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+    private final StructLike partition;
+
+    PositionDeletesWriterFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId,
+        StructLike partition) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+      this.partition = partition;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema = positionDeleteRowSchema();
+      StructType deleteSparkType = deleteSparkType();
+      StructType deleteSparkTypeWithoutRow = deleteSparkTypeWithoutRow();
+
+      SparkFileWriterFactory writerFactoryWithRow =
+          SparkFileWriterFactory.builderFor(table)
+              .dataSchema(writeSchema)

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");

Review Comment:
   Yep, done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentScanTask;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Joiner;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.util.StructLikeSet;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.write.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriteBuilder;
+import org.apache.spark.sql.types.StructType;
+
+/**
+ * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link
+ * SparkPositionDeletesRewrite}.
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewriteBuilder implements WriteBuilder {
+
+  private final SparkSession spark;
+  private final Table table;
+  private final SparkWriteConf writeConf;
+  private final LogicalWriteInfo writeInfo;
+  private final StructType dsSchema;
+  private final Schema writeSchema;
+
+  SparkPositionDeletesRewriteBuilder(
+      SparkSession spark, Table table, String branch, LogicalWriteInfo info) {
+    this.spark = spark;
+    this.table = table;
+    this.writeConf = new SparkWriteConf(spark, table, branch, info.options());
+    this.writeInfo = info;
+    this.dsSchema = info.schema();
+    this.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive());
+  }
+
+  @Override
+  public Write build() {
+    String fileSetId = writeConf.rewrittenFileSetId();
+
+    Preconditions.checkArgument(
+        fileSetId != null, "position_deletes table can only be written by RewriteDeleteFiles");
+    Preconditions.checkArgument(
+        writeConf.handleTimestampWithoutZone()
+            || !SparkUtil.hasTimestampWithoutZone(table.schema()),
+        SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR);
+
+    // all files of rewrite group have same partition and spec id
+    ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> tasks = taskSetManager.fetchTasks(table, fileSetId);
+    Preconditions.checkNotNull(tasks, "No scan tasks found for %s", fileSetId);
+    Preconditions.checkArgument(tasks.size() > 0, "No scan tasks found for %s", fileSetId);
+
+    int specId = specId(fileSetId, tasks);
+    StructLike partition = partition(fileSetId, tasks);
+
+    return new SparkPositionDeletesRewrite(
+        spark, table, writeConf, writeInfo, writeSchema, dsSchema, specId, partition);
+  }
+
+  private int specId(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    Set<Integer> specIds = tasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet());
+    Preconditions.checkArgument(
+        specIds.size() == 1,
+        "All scan tasks of %s are expected to have same spec id, but got %s",
+        fileSetId,
+        Joiner.on(",").join(specIds));
+    return tasks.get(0).spec().specId();
+  }
+
+  private StructLike partition(String fileSetId, List<PositionDeletesScanTask> tasks) {
+    StructLikeSet partitions = StructLikeSet.create(tasks.get(0).spec().partitionType());
+    partitions.addAll(tasks.stream().map(ContentScanTask::partition).collect(Collectors.toList()));
+    Preconditions.checkArgument(
+        partitions.size() == 1,
+        "All scan tasks of %s are expected to have the same partition",

Review Comment:
   Good catch, 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] szehon-ho commented on pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#issuecomment-1461120549

   Rebase on updated version of #6924 


-- 
This is an automated message from the Apache Git Service.
To 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 diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1137738090


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java:
##########
@@ -767,4 +824,270 @@ public void close() throws IOException {
       delegate.close();
     }
   }
+
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    private String fileSetID;
+
+    private PositionDeleteBatchWrite(String fileSetID) {
+      this.fileSetID = fileSetID;
+    }
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      if (cleanupOnAbort) {
+        SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+      } else {
+        LOG.warn("Skipping cleanup of written files");
+      }
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+              });
+
+      SparkFileWriterFactory writerFactoryWithRow =

Review Comment:
   Let me read up on this a bit.



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

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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1134386254


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*

Review Comment:
   This is just a refactor from: FileRewriteCoordinator (below)



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147884248


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/AbstractFileRewriteCoordinator.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ContentFile;
+import org.apache.iceberg.HasTableOperations;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractFileRewriteCoordinator<F extends ContentFile<F>> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(FileRewriteCoordinator.class);
+
+  private final Map<Pair<String, String>, Set<F>> resultMap = Maps.newConcurrentMap();
+
+  /**
+   * Called to persist the output of a rewrite action for a specific group. Since the write is done
+   * via a Spark Datasource, we have to propagate the result through this side-effect call.
+   *
+   * @param table table where the rewrite is occurring
+   * @param fileSetID the id used to identify the source set of files being rewritten
+   * @param newDataFiles the new files which have been written
+   */
+  public void stageRewrite(Table table, String fileSetID, Set<F> newDataFiles) {
+    LOG.debug(
+        "Staging the output for {} - fileset {} with {} files",
+        table.name(),
+        fileSetID,
+        newDataFiles.size());
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.put(id, newDataFiles);
+  }
+
+  public Set<F> fetchNewDataFiles(Table table, String fileSetID) {
+    Pair<String, String> id = toID(table, fileSetID);
+    Set<F> result = resultMap.get(id);
+    ValidationException.check(
+        result != null, "No results for rewrite of file set %s in table %s", fileSetID, table);
+
+    return result;
+  }
+
+  public void clearRewrite(Table table, String fileSetID) {
+    LOG.debug("Removing entry from RewriteCoordinator for {} - id {}", table.name(), fileSetID);
+    Pair<String, String> id = toID(table, fileSetID);
+    resultMap.remove(id);
+  }
+
+  public Set<String> fetchSetIDs(Table table) {
+    return resultMap.keySet().stream()
+        .filter(e -> e.first().equals(tableUUID(table)))
+        .map(Pair::second)
+        .collect(Collectors.toSet());
+  }
+
+  private Pair<String, String> toID(Table table, String setID) {

Review Comment:
   Done



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);

Review Comment:
   Removed



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}

Review Comment:
   Added period



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147887906


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema

Review Comment:
   As dicussed offline, I chose to make all columns in PositionDeletesTable real columns (there was some ramifications of using the column as partition column, and looks like it makes everything simpler)



-- 
This is an automated message from the Apache Git Service.
To 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] szehon-ho commented on a diff in pull request #7029: Spark 3.3: Dataset writes for position deletes

Posted by "szehon-ho (via GitHub)" <gi...@apache.org>.
szehon-ho commented on code in PR #7029:
URL: https://github.com/apache/iceberg/pull/7029#discussion_r1147890606


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =
+          new StructType(
+              new StructField[] {
+                dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()),
+                dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)

Review Comment:
   Changed this to have the correct one for without row



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -0,0 +1,429 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.PositionDeletesTable;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.deletes.PositionDelete;
+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.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkSchemaUtil;
+import org.apache.iceberg.spark.SparkWriteConf;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.StructProjection;
+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.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.LogicalWriteInfo;
+import org.apache.spark.sql.connector.write.PhysicalWriteInfo;
+import org.apache.spark.sql.connector.write.Write;
+import org.apache.spark.sql.connector.write.WriterCommitMessage;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
+ * {@link PositionDeleteBatchWrite}
+ *
+ * <p>This class is meant to be used for an action to rewrite delete files. Hence, it makes an
+ * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+ * from {@link ScanTaskSetManager}.
+ */
+public class SparkPositionDeletesRewrite implements Write {
+  private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeletesRewrite.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final String queryId;
+  private final FileFormat format;
+  private final long targetFileSize;
+  private final Schema writeSchema;
+  private final StructType dsSchema;
+  private final String fileSetId;
+  private final int specId;
+
+  /**
+   * Constructs a SparkPositionDeletesWrite.
+   *
+   * @param spark spark session
+   * @param table instance of {@link PositionDeletesTable}
+   * @param writeConf spark write config
+   * @param writeInfo spark write info
+   * @param writeSchema Iceberg output schema
+   * @param dsSchema schema of original incoming position deletes dataset
+   */
+  SparkPositionDeletesRewrite(
+      SparkSession spark,
+      Table table,
+      SparkWriteConf writeConf,
+      LogicalWriteInfo writeInfo,
+      Schema writeSchema,
+      StructType dsSchema) {
+    this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext());
+    this.table = table;
+    this.queryId = writeInfo.queryId();
+    this.format = writeConf.dataFileFormat();
+    this.targetFileSize = writeConf.targetDataFileSize();
+    this.writeSchema = writeSchema;
+    this.dsSchema = dsSchema;
+    this.fileSetId = writeConf.rewrittenFileSetId();
+
+    // all files of rewrite group have same spec id
+    ScanTaskSetManager scanTaskSetManager = ScanTaskSetManager.get();
+    List<PositionDeletesScanTask> scanTasks = scanTaskSetManager.fetchTasks(table, fileSetId);
+    this.specId = scanTasks.get(0).spec().specId();
+  }
+
+  @Override
+  public BatchWrite toBatch() {
+    return new PositionDeleteBatchWrite();
+  }
+
+  /** {@link BatchWrite} class for rewriting position deletes files from Spark */
+  class PositionDeleteBatchWrite implements BatchWrite {
+
+    @Override
+    public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) {
+      // broadcast the table metadata as the writer factory will be sent to executors
+      Broadcast<Table> tableBroadcast =
+          sparkContext.broadcast(SerializableTableWithSize.copyOf(table));
+      return new PositionDeltaWriteFactory(
+          tableBroadcast, queryId, format, targetFileSize, writeSchema, dsSchema, specId);
+    }
+
+    @Override
+    public void commit(WriterCommitMessage[] messages) {
+      PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get();
+      coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages)));
+    }
+
+    @Override
+    public void abort(WriterCommitMessage[] messages) {
+      SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages));
+    }
+
+    private List<DeleteFile> files(WriterCommitMessage[] messages) {
+      List<DeleteFile> files = Lists.newArrayList();
+
+      for (WriterCommitMessage message : messages) {
+        if (message != null) {
+          DeleteTaskCommit taskCommit = (DeleteTaskCommit) message;
+          files.addAll(Arrays.asList(taskCommit.files()));
+        }
+      }
+
+      return files;
+    }
+  }
+
+  /**
+   * Write factory for position deletes metadata table. Responsible for creating {@link
+   * DeleteWriter}.
+   *
+   * <p>This writer is meant to be used for an action to rewrite delete files. Hence, it makes an
+   * assumption that all incoming deletes belong to the same partition, and that incoming dataset is
+   * from {@link ScanTaskSetManager}.
+   */
+  static class PositionDeltaWriteFactory implements DataWriterFactory {
+    private final Broadcast<Table> tableBroadcast;
+    private final String queryId;
+    private final FileFormat format;
+    private final Long targetFileSize;
+    private final Schema writeSchema;
+    private final StructType dsSchema;
+    private final int specId;
+
+    PositionDeltaWriteFactory(
+        Broadcast<Table> tableBroadcast,
+        String queryId,
+        FileFormat format,
+        long targetFileSize,
+        Schema writeSchema,
+        StructType dsSchema,
+        int specId) {
+      this.tableBroadcast = tableBroadcast;
+      this.queryId = queryId;
+      this.format = format;
+      this.targetFileSize = targetFileSize;
+      this.writeSchema = writeSchema;
+      this.dsSchema = dsSchema;
+      this.specId = specId;
+    }
+
+    @Override
+    public DataWriter<InternalRow> createWriter(int partitionId, long taskId) {
+      Table table = tableBroadcast.value();
+
+      OutputFileFactory deleteFileFactory =
+          OutputFileFactory.builderFor(table, partitionId, taskId)
+              .format(format)
+              .operationId(queryId)
+              .suffix("deletes")
+              .build();
+
+      Schema positionDeleteRowSchema =
+          new Schema(
+              writeSchema
+                  .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME)
+                  .type()
+                  .asStructType()
+                  .fields());
+      StructType deleteFileType =

Review Comment:
   Changed



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

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

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


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