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/04/20 18:44:40 UTC

[GitHub] [iceberg] szehon-ho opened a new pull request, #7389: Spark 3.4: Implement rewrite position deletes

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

   This implements the RewritePositionDeleteFiles Interface (already existing) with a Spark action
   
   This action compacting/splitting position delete files, based on input parameters.  Most of the logic is re-used from RewriteDataFiles, via new Rewriter classes added in #7175 .  The additional logic here is sorting position deletes locally by 'file_path' and 'pos', as defined in Iceberg spec.
   
   This action will also notably remove 'dangling deletes', ie remove position deletes that no longer have a live data file.  Previously this was not possible in any Iceberg action.  This is implemented via a left semi-join on 'data_files' table.
   
   Remaining items: filter() is not yet supported.  As the position deletes rewrite is done against the position_deletes metadata table, the filter of data table does not apply.  Some work is needed to transform this.


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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.

Review Comment:
   It should be fine to have an empty set of new delete files, right?



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   After thinking more about, we can't rewrite equality deletes across sequence numbers.
   I created #7452 to add validation to `RewriteFiles`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -56,7 +56,7 @@
 
 /**
  * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
- * {@link PositionDeleteBatchWrite}.
+ * PositionDeleteBatchWrite.

Review Comment:
   Ah right, 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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +94,83 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
-    /** Returns the count of the position deletes that been rewritten. */
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
+    /** Returns the count of the position delete files that been rewritten. */
+    default int rewrittenDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::rewrittenDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the count of the added position delete files. */
+    default int addedDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::addedDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of position delete files that have been rewritten */
+    default long rewrittenBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::rewrittenBytesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of newly added position delete files */
+    default long addedBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::addedBytesCount)
+          .sum();
+    }
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {
+    /** Description of this position delete file group * */
+    PositionDeleteGroupInfo info();
+
+    /** Returns the count of the position delete files that been rewritten in this group. */
     int rewrittenDeleteFilesCount();
 
-    /** Returns the count of the added delete files. */
+    /** Returns the count of the added position delete files in this group. */
     int addedDeleteFilesCount();
+
+    /**
+     * Returns the number of bytes of position delete files that have been rewritten in this group
+     */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position delete files in this group */
+    long addedBytesCount();
+  }
+
+  /**
+   * A description of a position delete file group, when it was processed, and within which
+   * partition. For use tracking rewrite operations and for returning results.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupInfo {
+    /**
+     * returns which position delete file group this is out of the total set of file groups for this
+     * rewrite
+     */
+    int globalIndex();
+
+    /**
+     * returns which position delete file group this is out of the set of file groups for this
+     * partition
+     */
+    int partitionIndex();
+
+    /** returns which partition this position delete file group contains files from */
+    StructLike partition();

Review Comment:
   Added clarifying comment (its the coerced one)



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewritePositionDeleteFile Actions from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups fileSets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    RewriteFiles rewrite = table.newRewrite().validateFromSnapshot(startingSnapshotId);
+    rewrite.rewriteDeleteFiles(rewrittenDeleteFiles, addedDeleteFiles);
+
+    rewrite.commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should not
+   * throw any exceptions
+   *
+   * @param fileGroup group of files which has already been rewritten
+   */
+  public void abortFileGroup(RewritePositionDeleteGroup fileGroup) {
+    Preconditions.checkState(
+        fileGroup.addedDeleteFiles() != null, "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedDeleteFiles())

Review Comment:
   Shall we use bulk deletes if possible? Can we use `deleteFiles` in `CatalogUtil`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);

Review Comment:
   Changed.  It was copied it from RewriteDataFilesSparkAction, I guess we were very excited to commit here :p



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

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

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


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


[GitHub] [iceberg] ajantha-bhat commented on pull request #7389: Spark 3.4: Implement rewrite position deletes

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

   > Hi @ajantha-bhat sorry , actually @aokolnychyi pinged me and this should be part of the next release. So I will work on the procedure tomorrow at the first priority, if it is ok
   
   Ok. 


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = table.spec().partitionType();

Review Comment:
   How will this handle a case when there are deletes in multiple specs?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -56,7 +56,7 @@
 
 /**
  * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
- * {@link PositionDeleteBatchWrite}.
+ * PositionDeleteBatchWrite.

Review Comment:
   Fixes javadoc link warnings (seems javadocs cannot link to inner classes unfortunately)



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -56,7 +56,7 @@
 
 /**
  * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
- * {@link PositionDeleteBatchWrite}.
+ * PositionDeleteBatchWrite.

Review Comment:
   Fixes compiler javadoc link warnings (seems javadocs cannot link to inner classes unfortunately)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/SizeBasedPositionDeletesRewriter.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.List;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.util.PropertyUtil;
+
+public abstract class SizeBasedPositionDeletesRewriter
+    extends SizeBasedFileRewriter<PositionDeletesScanTask, DeleteFile> {
+
+  protected SizeBasedPositionDeletesRewriter(Table table) {
+    super(table);
+  }
+
+  @Override
+  protected Iterable<PositionDeletesScanTask> filterFiles(Iterable<PositionDeletesScanTask> tasks) {
+    return Iterables.filter(tasks, this::wronglySized);
+  }
+
+  @Override
+  protected Iterable<List<PositionDeletesScanTask>> filterFileGroups(
+      List<List<PositionDeletesScanTask>> groups) {
+    return Iterables.filter(groups, this::shouldRewrite);
+  }
+
+  private boolean shouldRewrite(List<PositionDeletesScanTask> group) {
+    return enoughInputFiles(group) || enoughContent(group) || tooMuchContent(group);
+  }
+
+  @Override
+  protected long defaultTargetFileSize() {
+    return PropertyUtil.propertyAsLong(
+        table().properties(),
+        TableProperties.WRITE_TARGET_FILE_SIZE_BYTES,

Review Comment:
   Shouldn't we use `DELETE_TARGET_FILE_SIZE_BYTES`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,
+      List<PositionDeletesScanTask> fileScanTasks) {
+    this.info = info;
+    this.positionDeletesScanTasks = fileScanTasks;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupInfo info() {
+    return info;
+  }
+
+  public List<PositionDeletesScanTask> scans() {
+    return positionDeletesScanTasks;
+  }
+
+  public void setOutputFiles(Set<DeleteFile> files) {
+    addedDeleteFiles = files;
+  }
+
+  public Set<DeleteFile> rewrittenDeleteFiles() {
+    return scans().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet());
+  }
+
+  public Set<DeleteFile> addedDeleteFiles() {
+    return addedDeleteFiles;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult asResult() {
+    Preconditions.checkState(
+        addedDeleteFiles != null, "Cannot get result, Group was never rewritten");
+    long addedBytes = addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();
+
+    return ImmutablePositionDeleteGroupRewriteResult.builder()
+        .info(info)
+        .addedDeleteFilesCount(addedDeleteFiles.size())
+        .rewrittenDeleteFilesCount(positionDeletesScanTasks.size())
+        .rewrittenBytesCount(rewrittenBytes())
+        .addedBytesCount(addedBytes())
+        .build();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("info", info)
+        .add("numRewrittenPositionDeleteFiles", positionDeletesScanTasks.size())
+        .add(
+            "numAddedPositionDeleteFiles",
+            addedDeleteFiles == null
+                ? "Rewrite Incomplete"
+                : Integer.toString(addedDeleteFiles.size()))
+        .add("numRewrittenBytes", rewrittenBytes())

Review Comment:
   Do we have to include `numAddedBytes`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,
+      List<PositionDeletesScanTask> fileScanTasks) {
+    this.info = info;
+    this.positionDeletesScanTasks = fileScanTasks;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupInfo info() {
+    return info;
+  }
+
+  public List<PositionDeletesScanTask> scans() {
+    return positionDeletesScanTasks;
+  }
+
+  public void setOutputFiles(Set<DeleteFile> files) {
+    addedDeleteFiles = files;
+  }
+
+  public Set<DeleteFile> rewrittenDeleteFiles() {
+    return scans().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet());
+  }
+
+  public Set<DeleteFile> addedDeleteFiles() {
+    return addedDeleteFiles;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult asResult() {
+    Preconditions.checkState(
+        addedDeleteFiles != null, "Cannot get result, Group was never rewritten");
+    long addedBytes = addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();

Review Comment:
   Redundant line? We use `addedBytes()` directly.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large

Review Comment:
   optional: Javadoc points to itself.



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

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

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


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


[GitHub] [iceberg] ajantha-bhat commented on pull request #7389: Spark 3.4: Implement rewrite position deletes

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

   @szehon-ho: Are you already working on the CALL procedure for the same? If not, I would like to work on it.  


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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriter.java:
##########
@@ -69,14 +69,4 @@ private DistributionMode distributionMode(List<FileScanTask> group) {
     boolean requiresRepartition = !group.get(0).spec().equals(table().spec());
     return requiresRepartition ? DistributionMode.RANGE : DistributionMode.NONE;
   }
-
-  /**
-   * Returns the smallest of our max write file threshold and our estimated split size based on the
-   * number of output files we want to generate. Add an overhead onto the estimated split size to
-   * try to avoid small errors in size creating brand-new files.
-   */
-  private long splitSize(long inputSize) {

Review Comment:
   Moved to common base class to re-use the code.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes

Review Comment:
   If we decide to keep this config and default value here, let's switch the order of constants, it is easier to read.
   
   ```
   100 * 1024L * 1024L * 1024L
   ```



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes

Review Comment:
   If we decide to keep this config and default value here, let's switch the order of constants, it is easier to read.
   
   ```
   100 * 1024L * 1024L * 1024L // 100 GB
   ```



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Did you add the required annotation on the outer class? I forgot the name of it.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   Let's not add an overloaded method here. We will rework this API shortly after. For now, let's use the method that accepts 4 sets. This PR should use `RewriteFiles` as is.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,
+      List<PositionDeletesScanTask> fileScanTasks) {
+    this.info = info;
+    this.positionDeletesScanTasks = fileScanTasks;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupInfo info() {
+    return info;
+  }
+
+  public List<PositionDeletesScanTask> scans() {
+    return positionDeletesScanTasks;
+  }
+
+  public void setOutputFiles(Set<DeleteFile> files) {
+    addedDeleteFiles = files;
+  }
+
+  public Set<DeleteFile> rewrittenDeleteFiles() {
+    return scans().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet());
+  }
+
+  public Set<DeleteFile> addedDeleteFiles() {
+    return addedDeleteFiles;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult asResult() {
+    Preconditions.checkState(
+        addedDeleteFiles != null, "Cannot get result, Group was never rewritten");
+    long addedBytes = addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();
+
+    return ImmutablePositionDeleteGroupRewriteResult.builder()
+        .info(info)
+        .addedDeleteFilesCount(addedDeleteFiles.size())
+        .rewrittenDeleteFilesCount(positionDeletesScanTasks.size())
+        .rewrittenBytesCount(rewrittenBytes())
+        .addedBytesCount(addedBytes())
+        .build();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("info", info)
+        .add("numRewrittenPositionDeleteFiles", positionDeletesScanTasks.size())
+        .add(
+            "numAddedPositionDeleteFiles",
+            addedDeleteFiles == null
+                ? "Rewrite Incomplete"
+                : Integer.toString(addedDeleteFiles.size()))
+        .add("numRewrittenBytes", rewrittenBytes())
+        .toString();
+  }
+
+  public long rewrittenBytes() {
+    return positionDeletesScanTasks.stream().mapToLong(PositionDeletesScanTask::length).sum();
+  }
+
+  public long addedBytes() {
+    return addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();
+  }
+
+  public int numDeleteFiles() {

Review Comment:
   minor: The name is too generic given that we have added and rewritten files.



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,
+      List<PositionDeletesScanTask> fileScanTasks) {
+    this.info = info;
+    this.positionDeletesScanTasks = fileScanTasks;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupInfo info() {
+    return info;
+  }
+
+  public List<PositionDeletesScanTask> scans() {
+    return positionDeletesScanTasks;
+  }
+
+  public void setOutputFiles(Set<DeleteFile> files) {
+    addedDeleteFiles = files;
+  }
+
+  public Set<DeleteFile> rewrittenDeleteFiles() {
+    return scans().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet());
+  }
+
+  public Set<DeleteFile> addedDeleteFiles() {
+    return addedDeleteFiles;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult asResult() {
+    Preconditions.checkState(
+        addedDeleteFiles != null, "Cannot get result, Group was never rewritten");
+    long addedBytes = addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();
+
+    return ImmutablePositionDeleteGroupRewriteResult.builder()
+        .info(info)
+        .addedDeleteFilesCount(addedDeleteFiles.size())
+        .rewrittenDeleteFilesCount(positionDeletesScanTasks.size())
+        .rewrittenBytesCount(rewrittenBytes())
+        .addedBytesCount(addedBytes())
+        .build();
+  }
+
+  @Override
+  public String toString() {
+    return MoreObjects.toStringHelper(this)
+        .add("info", info)
+        .add("numRewrittenPositionDeleteFiles", positionDeletesScanTasks.size())
+        .add(
+            "numAddedPositionDeleteFiles",
+            addedDeleteFiles == null
+                ? "Rewrite Incomplete"
+                : Integer.toString(addedDeleteFiles.size()))
+        .add("numRewrittenBytes", rewrittenBytes())
+        .toString();
+  }
+
+  public long rewrittenBytes() {
+    return positionDeletesScanTasks.stream().mapToLong(PositionDeletesScanTask::length).sum();
+  }
+
+  public long addedBytes() {
+    return addedDeleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum();
+  }
+
+  public int numDeleteFiles() {

Review Comment:
   minor: The method name is too generic given that we have added and rewritten files.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();

Review Comment:
   Shall we add default implementations to aggregate the individual results?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */

Review Comment:
   The doc in all these 4 methods should use `position delete files` rather than `position deletes` since these methods represent file counts.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
.palantir/revapi.yml:
##########
@@ -417,6 +417,7 @@ acceptedBreaks:
     - code: "java.method.addedToInterface"
       new: "method java.lang.String org.apache.iceberg.view.ViewVersion::operation()"
       justification: "Add operation API to view version"
+

Review Comment:
   Is this needed?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,55 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The max number of file groups to be simultaneously rewritten by the rewrite strategy. The
+   * structure and contents of the group is determined by the rewrite strategy. Each file group will
+   * be rewritten independently and asynchronously.
+   */
+  String MAX_CONCURRENT_FILE_GROUP_REWRITES = "max-concurrent-file-group-rewrites";
+
+  int MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT = 5;
+
+  /**
+   * Forces the rewrite job order based on the value.
+   *
+   * <p>

Review Comment:
   question: Does this `<p>` make any difference in the actual doc?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +94,83 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
-    /** Returns the count of the position deletes that been rewritten. */
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
+    /** Returns the count of the position delete files that been rewritten. */
+    default int rewrittenDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::rewrittenDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the count of the added position delete files. */
+    default int addedDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::addedDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of position delete files that have been rewritten */
+    default long rewrittenBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::rewrittenBytesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of newly added position delete files */
+    default long addedBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::addedBytesCount)
+          .sum();
+    }
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {
+    /** Description of this position delete file group * */
+    PositionDeleteGroupInfo info();
+
+    /** Returns the count of the position delete files that been rewritten in this group. */
     int rewrittenDeleteFilesCount();
 
-    /** Returns the count of the added delete files. */
+    /** Returns the count of the added position delete files in this group. */
     int addedDeleteFilesCount();
+
+    /**
+     * Returns the number of bytes of position delete files that have been rewritten in this group

Review Comment:
   optional: Just `of rewritten position delete files in this group` to mirror the sentence below?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.PositionDeleteGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;

Review Comment:
   optional: Just `tasks`?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by {@link RewritePositionDeleteFiles} from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups file sets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    RewriteFiles rewrite = table.newRewrite().validateFromSnapshot(startingSnapshotId);

Review Comment:
   optional: Combine all in one call?
   
   ```
   table
       .newRewrite()
       .validateFromSnapshot(startingSnapshotId)
       .rewriteFiles(ImmutableSet.of(), rewrittenDeleteFiles, ImmutableSet.of(), addedDeleteFiles)
       .commit();
   ```
   
   



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by {@link RewritePositionDeleteFiles} from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups file sets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    RewriteFiles rewrite = table.newRewrite().validateFromSnapshot(startingSnapshotId);
+    rewrite.rewriteFiles(
+        ImmutableSet.of(), rewrittenDeleteFiles, ImmutableSet.of(), addedDeleteFiles);
+
+    rewrite.commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should not
+   * throw any exceptions

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



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);

Review Comment:
   optional: What about a direct static import for `POSITION_DELETES`?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =

Review Comment:
   optional: Just `tasks`?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting

Review Comment:
   Is it actually being used in tests?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {
+    switch (rewriteJobOrder) {
+      case BYTES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::rewrittenBytes);
+      case BYTES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::rewrittenBytes, Comparator.reverseOrder());
+      case FILES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::numRewrittenDeleteFiles);
+      case FILES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::numRewrittenDeleteFiles, Comparator.reverseOrder());
+      default:
+        return (fileGroupOne, fileGroupTwo) -> 0;
+    }
+  }
+
+  void validateAndInitOptions() {

Review Comment:
   Why not private?
   



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +94,83 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
-    /** Returns the count of the position deletes that been rewritten. */
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
+    /** Returns the count of the position delete files that been rewritten. */
+    default int rewrittenDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::rewrittenDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the count of the added position delete files. */
+    default int addedDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::addedDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of position delete files that have been rewritten */
+    default long rewrittenBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::rewrittenBytesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of newly added position delete files */
+    default long addedBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::addedBytesCount)
+          .sum();
+    }
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {
+    /** Description of this position delete file group * */
+    PositionDeleteGroupInfo info();
+
+    /** Returns the count of the position delete files that been rewritten in this group. */
     int rewrittenDeleteFilesCount();
 
-    /** Returns the count of the added delete files. */
+    /** Returns the count of the added position delete files in this group. */
     int addedDeleteFilesCount();
+
+    /**
+     * Returns the number of bytes of position delete files that have been rewritten in this group
+     */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position delete files in this group */
+    long addedBytesCount();
+  }
+
+  /**
+   * A description of a position delete file group, when it was processed, and within which
+   * partition. For use tracking rewrite operations and for returning results.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupInfo {
+    /**
+     * returns which position delete file group this is out of the total set of file groups for this

Review Comment:
   Start with a capital letter and end with `.` to be consistent with the doc of the interface above?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +94,83 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
-    /** Returns the count of the position deletes that been rewritten. */
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
+    /** Returns the count of the position delete files that been rewritten. */
+    default int rewrittenDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::rewrittenDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the count of the added position delete files. */
+    default int addedDeleteFilesCount() {
+      return rewriteResults().stream()
+          .mapToInt(PositionDeleteGroupRewriteResult::addedDeleteFilesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of position delete files that have been rewritten */
+    default long rewrittenBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::rewrittenBytesCount)
+          .sum();
+    }
+
+    /** Returns the number of bytes of newly added position delete files */
+    default long addedBytesCount() {
+      return rewriteResults().stream()
+          .mapToLong(PositionDeleteGroupRewriteResult::addedBytesCount)
+          .sum();
+    }
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {
+    /** Description of this position delete file group * */
+    PositionDeleteGroupInfo info();
+
+    /** Returns the count of the position delete files that been rewritten in this group. */
     int rewrittenDeleteFilesCount();
 
-    /** Returns the count of the added delete files. */
+    /** Returns the count of the added position delete files in this group. */
     int addedDeleteFilesCount();
+
+    /**
+     * Returns the number of bytes of position delete files that have been rewritten in this group
+     */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position delete files in this group */
+    long addedBytesCount();
+  }
+
+  /**
+   * A description of a position delete file group, when it was processed, and within which
+   * partition. For use tracking rewrite operations and for returning results.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupInfo {
+    /**
+     * returns which position delete file group this is out of the total set of file groups for this
+     * rewrite
+     */
+    int globalIndex();
+
+    /**
+     * returns which position delete file group this is out of the set of file groups for this
+     * partition
+     */
+    int partitionIndex();
+
+    /** returns which partition this position delete file group contains files from */
+    StructLike partition();

Review Comment:
   question: Is this the original partition or the adapted partition?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.PositionDeleteGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.PositionDeleteGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      PositionDeleteGroupInfo info, List<PositionDeletesScanTask> fileScanTasks) {

Review Comment:
   optional: Just `tasks`?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +94,83 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable

Review Comment:
   I think we would need `@Value.Enclosing` on the parent class to avoid naming conflicts.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */

Review Comment:
   minor: Just `{@link RewritePositionDeleteFiles}` as we already have an import for that interface?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)

Review Comment:
   Duplicate calls?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Yeah, that's the annotation I meant. Even if we keep the existing name, we would have to add it to the parent class.
   
   My initial idea was to use the exact same naming as in `RewriteDataFiles` for consistency. That would be fine as these interfaces are nested. For instance, we always call it `Result` as the actual class name is `XXX.Result` where the parent interface adds the required context. I believe the same could be applied to other interfaces.
   
   ```
   RewritePositionDeleteFiles.Result
   RewritePositionDeleteFiles.FileGroupRewriteResult
   RewritePositionDeleteFiles.FileGroupInfo
   ```
   
   I am no longer sure, maybe we don't have to follow `RewriteDataFiles` and what you have is better. Let me think, let's keep it as is for now, just add the annotation to the parent class.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {

Review Comment:
   What about having a method in `RewritePositionDeleteGroup` that accepts `RewriteJobOrder` and returns a comparator to simplify the action, which is already complicated?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);

Review Comment:
   My bad, I overlooked.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());

Review Comment:
   What do you think about moving `cloneSession()` and disable AQE call to the rewriter? I don't think we need to disable AQE in the action itself anymore. It should be more obvious and reliable to clone the session in the rewriter as it is the only one that depends on a specific write distribution.
   
   We should do the same for data later.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesBinPackRewriter.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+
+class SparkPositionDeletesBinPackRewriter extends SizeBasedPositionDeletesRewriter {

Review Comment:
   optional: `SparkBinPackPositionDeletesRewriter` to match `SparkBinPackDataRewriter`?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesBinPackRewriter.java:
##########
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+
+class SparkPositionDeletesBinPackRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesBinPackRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {

Review Comment:
   I would say it should describe the algorithm, so I'd return `BIN-PACK` for now like in data rewriter. We can add more details and make it more helpful in the future.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();

Review Comment:
   What do you think about adding an EMPTY constant to the interface and using it?
   
   ```
   interface Result {
     Result EMPTY = ImmutableXXX.builder().build();
   }
   ```
   
   ```
   if (table.currentSnapshot() == null) {
     LOG.info("Nothing found to rewrite in empty table {}", table.name());
     return Result.EMPTY;
   }
   ```



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);

Review Comment:
   minor: Why nouns start with capital letters?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(

Review Comment:
   What about an explicit for loop? Spotless adds extra padding due to its formatting.
   
   ```
   for (PositionDeletesScanTask task : tasks) {
     ...
   }
   ```



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());

Review Comment:
   minor: Reuse `taskPartition`?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =

Review Comment:
   optional: What about a direct import to shorten the line?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =

Review Comment:
   We shouldn't use a regular map with `StructLike` keys. We should either use `StructLikeMap` (which would mean we have to report the adapted partition tuple) or add a class `PartitionMap` similar to `PartitionSet` that would keep track of spec ID and partition (would allow us to report the original partition).



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {

Review Comment:
   Why not private?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {
+    switch (rewriteJobOrder) {
+      case BYTES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::rewrittenBytes);
+      case BYTES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::rewrittenBytes, Comparator.reverseOrder());
+      case FILES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::numRewrittenDeleteFiles);
+      case FILES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::numRewrittenDeleteFiles, Comparator.reverseOrder());
+      default:
+        return (fileGroupOne, fileGroupTwo) -> 0;
+    }
+  }
+
+  void validateAndInitOptions() {
+    Set<String> validOptions = Sets.newHashSet(rewriter.validOptions());
+    validOptions.addAll(VALID_OPTIONS);
+
+    Set<String> invalidKeys = Sets.newHashSet(options().keySet());
+    invalidKeys.removeAll(validOptions);
+
+    Preconditions.checkArgument(
+        invalidKeys.isEmpty(),
+        "Cannot use options %s, they are not supported by the action or the rewriter %s",
+        invalidKeys,
+        rewriter.description());
+
+    rewriter.init(options());
+
+    maxConcurrentFileGroupRewrites =

Review Comment:
   optional: Sometimes, we include `this.XXX` to indicate we assign a value to an instance variable. Up to you.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {
+    switch (rewriteJobOrder) {
+      case BYTES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::rewrittenBytes);
+      case BYTES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::rewrittenBytes, Comparator.reverseOrder());
+      case FILES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::numRewrittenDeleteFiles);
+      case FILES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::numRewrittenDeleteFiles, Comparator.reverseOrder());
+      default:
+        return (fileGroupOne, fileGroupTwo) -> 0;
+    }
+  }
+
+  void validateAndInitOptions() {
+    Set<String> validOptions = Sets.newHashSet(rewriter.validOptions());
+    validOptions.addAll(VALID_OPTIONS);
+
+    Set<String> invalidKeys = Sets.newHashSet(options().keySet());
+    invalidKeys.removeAll(validOptions);
+
+    Preconditions.checkArgument(
+        invalidKeys.isEmpty(),
+        "Cannot use options %s, they are not supported by the action or the rewriter %s",
+        invalidKeys,
+        rewriter.description());
+
+    rewriter.init(options());
+
+    maxConcurrentFileGroupRewrites =
+        PropertyUtil.propertyAsInt(
+            options(),
+            MAX_CONCURRENT_FILE_GROUP_REWRITES,
+            MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT);
+
+    maxCommits =
+        PropertyUtil.propertyAsInt(
+            options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    partialProgressEnabled =
+        PropertyUtil.propertyAsBoolean(
+            options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    rewriteJobOrder =
+        RewriteJobOrder.fromName(
+            PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT));
+
+    Preconditions.checkArgument(
+        maxConcurrentFileGroupRewrites >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_REWRITES,
+        maxConcurrentFileGroupRewrites);
+
+    Preconditions.checkArgument(
+        !partialProgressEnabled || maxCommits > 0,
+        "Cannot set %s to %s, the value must be positive when %s is true",
+        PARTIAL_PROGRESS_MAX_COMMITS,
+        maxCommits,
+        PARTIAL_PROGRESS_ENABLED);
+  }
+
+  private String jobDesc(RewritePositionDeleteGroup group, RewriteExecutionContext ctx) {
+    StructLike partition = group.info().partition();
+    if (partition.size() > 0) {
+      return String.format(
+          "Rewriting %d position delete files (%s, file group %d/%d, %s (%d/%d)) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          partition,
+          group.info().partitionIndex(),
+          ctx.groupsInPartition(partition),
+          table.name());
+    } else {
+      return String.format(
+          "Rewriting %d position files (%s, file group %d/%d) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          table.name());
+    }
+  }
+
+  @VisibleForTesting
+  static class RewriteExecutionContext {
+    private final Map<StructLike, Integer> numGroupsByPartition;
+    private final int totalGroupCount;
+    private final Map<StructLike, Integer> partitionIndexMap;
+    private final AtomicInteger groupIndex;
+
+    RewriteExecutionContext(
+        Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+      this.numGroupsByPartition =
+          groupsByPartition.entrySet().stream()
+              .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size()));
+      this.totalGroupCount = numGroupsByPartition.values().stream().reduce(Integer::sum).orElse(0);
+      this.partitionIndexMap = Maps.newConcurrentMap();
+      this.groupIndex = new AtomicInteger(1);
+    }
+
+    public int currentGlobalIndex() {
+      return groupIndex.getAndIncrement();
+    }
+
+    public int currentPartitionIndex(StructLike partition) {
+      return partitionIndexMap.merge(partition, 1, Integer::sum);
+    }
+
+    public int groupsInPartition(StructLike partition) {
+      return numGroupsByPartition.get(partition);
+    }
+
+    public int totalGroupCount() {
+      return totalGroupCount;
+    }
+  }
+
+  private static class PartitionScans {
+    private final StructLike originalPartition;
+    private List<PositionDeletesScanTask> scanTasks = Lists.newArrayList();

Review Comment:
   Can be final?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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

   Hi @ajantha-bhat sorry , actually @aokolnychyi pinged me and this should be part of the next release.  So I will work on the procedure tomorrow at the first priority.


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   After thinking more about, we can't rewrite equality deletes across sequence numbers.
   I created #7452 to address that.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";
+
+  long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 1024L * 1024L * 1024L * 100L; // 100 Gigabytes
+
+  /**
+   * The max number of file groups to be simultaneously rewritten by the rewrite strategy. The
+   * structure and contents of the group is determined by the rewrite strategy. Each file group will
+   * be rewritten independently and asynchronously.
+   */
+  String MAX_CONCURRENT_FILE_GROUP_REWRITES = "max-concurrent-file-group-rewrites";
+
+  int MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT = 5;
+
+  /**
+   * The output file size that this rewrite strategy will attempt to generate when rewriting files.
+   * By default this will use the "write.target-file-size-bytes value" in the table properties of

Review Comment:
   I believe this is also part of the rewriter API now.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */

Review Comment:
   The doc in all these 4 methods must use `position delete files` rather than `position deletes` since these methods represent file counts.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {

Review Comment:
   Would it be fair to say we bin-pack position deletes here? If so, what about aligning the name with data compaction?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";

Review Comment:
   Let me think.



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);

Review Comment:
   This is creating a PositionDeletes table object, not loading rows into Dataset



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   We should also reconsider how this API looks like. Let me think.



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -32,13 +32,19 @@ public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /** @deprecated will be removed in 1.4.0; use {@link #fetchNewFiles(Table, String)} instead. */
+  /**
+   * @deprecated will be removed in 1.4.0; use {@link
+   *     BaseFileRewriteCoordinator#fetchNewFiles(Table, String)} instead.

Review Comment:
   Fixes compiler javadoc link warnings



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   Probably, there is a problem in `RewriteFiles` right now. I think this API would assign new delete files a brand new data sequence number while we should use the max data sequence number of all rewritten position deletes.
   
   On a side note, I am not sure we can ever rewrite equality deletes across sequence numbers. Let me think.



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Oh I see, well I think the problem is not that (if you mean @Enclosing @Immutable) , they're not enclosing each other.  it's just they all end up in the same place.  That being said, FileGroupRewriteResult seems to me a bit particular to RewriteDataFiles somehow



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {

Review Comment:
   Analogy of RewriteFileGroup



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   I see, used that annotation and it does not collide anymore.  Made the names match RewriteDataFiles.Result classes



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {
+    switch (rewriteJobOrder) {
+      case BYTES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::rewrittenBytes);
+      case BYTES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::rewrittenBytes, Comparator.reverseOrder());
+      case FILES_ASC:
+        return Comparator.comparing(RewritePositionDeleteGroup::numRewrittenDeleteFiles);
+      case FILES_DESC:
+        return Comparator.comparing(
+            RewritePositionDeleteGroup::numRewrittenDeleteFiles, Comparator.reverseOrder());
+      default:
+        return (fileGroupOne, fileGroupTwo) -> 0;
+    }
+  }
+
+  void validateAndInitOptions() {
+    Set<String> validOptions = Sets.newHashSet(rewriter.validOptions());
+    validOptions.addAll(VALID_OPTIONS);
+
+    Set<String> invalidKeys = Sets.newHashSet(options().keySet());
+    invalidKeys.removeAll(validOptions);
+
+    Preconditions.checkArgument(
+        invalidKeys.isEmpty(),
+        "Cannot use options %s, they are not supported by the action or the rewriter %s",
+        invalidKeys,
+        rewriter.description());
+
+    rewriter.init(options());
+
+    maxConcurrentFileGroupRewrites =
+        PropertyUtil.propertyAsInt(
+            options(),
+            MAX_CONCURRENT_FILE_GROUP_REWRITES,
+            MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT);
+
+    maxCommits =
+        PropertyUtil.propertyAsInt(
+            options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    partialProgressEnabled =
+        PropertyUtil.propertyAsBoolean(
+            options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    rewriteJobOrder =
+        RewriteJobOrder.fromName(
+            PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT));
+
+    Preconditions.checkArgument(
+        maxConcurrentFileGroupRewrites >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_REWRITES,
+        maxConcurrentFileGroupRewrites);
+
+    Preconditions.checkArgument(
+        !partialProgressEnabled || maxCommits > 0,
+        "Cannot set %s to %s, the value must be positive when %s is true",
+        PARTIAL_PROGRESS_MAX_COMMITS,
+        maxCommits,
+        PARTIAL_PROGRESS_ENABLED);
+  }
+
+  private String jobDesc(RewritePositionDeleteGroup group, RewriteExecutionContext ctx) {
+    StructLike partition = group.info().partition();
+    if (partition.size() > 0) {
+      return String.format(
+          "Rewriting %d position delete files (%s, file group %d/%d, %s (%d/%d)) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          partition,
+          group.info().partitionIndex(),
+          ctx.groupsInPartition(partition),
+          table.name());
+    } else {
+      return String.format(
+          "Rewriting %d position files (%s, file group %d/%d) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          table.name());
+    }
+  }
+
+  @VisibleForTesting
+  static class RewriteExecutionContext {
+    private final Map<StructLike, Integer> numGroupsByPartition;
+    private final int totalGroupCount;
+    private final Map<StructLike, Integer> partitionIndexMap;
+    private final AtomicInteger groupIndex;
+
+    RewriteExecutionContext(
+        Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+      this.numGroupsByPartition =
+          groupsByPartition.entrySet().stream()
+              .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().size()));
+      this.totalGroupCount = numGroupsByPartition.values().stream().reduce(Integer::sum).orElse(0);
+      this.partitionIndexMap = Maps.newConcurrentMap();
+      this.groupIndex = new AtomicInteger(1);
+    }
+
+    public int currentGlobalIndex() {
+      return groupIndex.getAndIncrement();
+    }
+
+    public int currentPartitionIndex(StructLike partition) {
+      return partitionIndexMap.merge(partition, 1, Integer::sum);
+    }
+
+    public int groupsInPartition(StructLike partition) {
+      return numGroupsByPartition.get(partition);
+    }
+
+    public int totalGroupCount() {
+      return totalGroupCount;
+    }
+  }
+
+  private static class PartitionScans {
+    private final StructLike originalPartition;
+    private List<PositionDeletesScanTask> scanTasks = Lists.newArrayList();

Review Comment:
   Removed this class actually as part of resolving another 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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark
+            .read()
+            .format("iceberg")
+            .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId)
+            .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
+            .option(SparkReadOptions.FILE_OPEN_COST, "0")
+            .load(groupId);
+
+    // keep only valid position deletes
+    Dataset<Row> dataFiles = dataFiles(partitionType, partition);
+    Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path"));
+    Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi");
+
+    // write the packed deletes into new files where each split becomes a new file
+    joined
+        .sortWithinPartitions("file_path", "pos")
+        .write()
+        .format("iceberg")
+        .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId)
+        .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+        .mode("append")
+        .save("default.test_table" + ".position_deletes");
+  }
+
+  /** Returns entries of {@link DataFilesTable} of specified partition */
+  private Dataset<Row> dataFiles(Types.StructType partitionType, StructLike partition) {
+    List<Types.NestedField> fields = partitionType.fields();
+    Optional<Column> condition =
+        IntStream.range(0, fields.size())
+            .mapToObj(
+                i -> {
+                  Class<?> type = fields.get(i).type().typeId().javaClass();
+                  Object value = partition.get(i, type);
+                  Column col = col("partition." + fields.get(i).name());
+                  return col.equalTo(value);
+                })
+            .reduce(Column::and);
+    if (condition.isPresent()) {
+      return SparkTableUtil.loadMetadataTable(spark, table(), MetadataTableType.DATA_FILES)

Review Comment:
   We also need to ensure we scan the same version of the table...



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Immutables generate all of these into the same package (org.apache.iceberg.actions...), so this collides with the RewriteFile one (although technically, it is the same object..)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +93,80 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
-    /** Returns the count of the position deletes that been rewritten. */
+    List<FileGroupRewriteResult> rewriteResults();
+
+    /** Returns the count of the position delete files that been rewritten. */

Review Comment:
   typo: `been rewritten` -> `have been rewritten`?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final FileGroupInfo info;
+  private final List<PositionDeletesScanTask> tasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      FileGroupInfo info, List<PositionDeletesScanTask> fileScanTasks) {

Review Comment:
   minor: `fileScanTasks` -> `tasks` to stay on one line?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by {@link RewritePositionDeleteFiles} from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups

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



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by {@link RewritePositionDeleteFiles} from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups file sets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {

Review Comment:
   optional: I'd either call these methods `commit` and `abort` or rename `commitOrClean` to include `FileGroup` as the naming is inconsistent. Up to you, I know we have this naming for rewriting data files.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackDataRewriter.java:
##########
@@ -69,14 +69,4 @@ private DistributionMode distributionMode(List<FileScanTask> group) {
     boolean requiresRepartition = !group.get(0).spec().equals(table().spec());
     return requiresRepartition ? DistributionMode.RANGE : DistributionMode.NONE;
   }
-
-  /**
-   * Returns the smallest of our max write file threshold and our estimated split size based on the
-   * number of output files we want to generate. Add an overhead onto the estimated split size to
-   * try to avoid small errors in size creating brand-new files.
-   */
-  private long splitSize(long inputSize) {

Review Comment:
   Shall we drop `SPLIT_OVERHEAD` in this class as well?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+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.Sets;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by {@link RewritePositionDeleteFiles} from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups file sets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    table
+        .newRewrite()
+        .validateFromSnapshot(startingSnapshotId)
+        .rewriteFiles(ImmutableSet.of(), rewrittenDeleteFiles, ImmutableSet.of(), addedDeleteFiles)
+        .commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should not
+   * throw any exceptions.
+   *
+   * @param fileGroup group of files which has already been rewritten
+   */
+  public void abortFileGroup(RewritePositionDeleteGroup fileGroup) {
+    Preconditions.checkState(
+        fileGroup.addedDeleteFiles() != null, "Cannot abort a fileGroup that was not rewritten");
+
+    Set<String> filePaths =

Review Comment:
   minor: We could use `Iterables.transform()` to avoid an eager set materialization.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY = ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY;
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike taskPartition = task.file().partition();
+        StructLike coerced =
+            PartitionUtil.coercePartition(partitionType, task.spec(), taskPartition);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition

Review Comment:
   Hm, shouldn't we use the adapted partition here as well?



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -18,16 +18,68 @@
  */
 package org.apache.iceberg.actions;
 
+import java.util.List;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
+import org.immutables.value.Value;
 
 /**
  * An action for rewriting position delete files.
  *
  * <p>Generally used for optimizing the size and layout of position delete files within a table.
  */
+@Value.Enclosing
 public interface RewritePositionDeleteFiles

Review Comment:
   This interface looks good to me. Shall we deprecate `RewritePositionDeleteStrategy`, btw?



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {

Review Comment:
   What's the naming convention here? You are using `RewritePositionDeletesXXX` (plural) in other places.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY = ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY;
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike taskPartition = task.file().partition();

Review Comment:
   optional: You may consider a helper method.
   
   ```
   private StructLike XXX(PositionDeletesScanTask task, StructType partitionType) {
     return PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
   }
   ```



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY = ImmutableRewritePositionDeleteFiles.Result.builder().build();

Review Comment:
   optional: `EMPTY_RESULT`?



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java:
##########
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import static org.apache.iceberg.MetadataTableType.POSITION_DELETES;
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+
+class SparkBinPackPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkBinPackPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    this.spark = spark.cloneSession();
+    this.spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+  }
+
+  @Override
+  public String description() {
+    return "BIN-PACK";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table(), POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // ensure AQE is disabled for full control of splits

Review Comment:
   This is probably redundant now given that we clone the session in this class.



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

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

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


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


[GitHub] [iceberg] chenwyi2 commented on pull request #7389: Spark 3.4: Implement rewrite position deletes

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

   when i backport to spark 3.1 with error:
   java.lang.IllegalArgumentException: Cannot parse path or identifier: 9a439584-c00f-45a5-9df7-31adfe182900
   	at org.apache.iceberg.spark.Spark3Util.catalogAndIdentifier(Spark3Util.java:722)
   	at org.apache.iceberg.spark.Spark3Util.catalogAndIdentifier(Spark3Util.java:713)
   	at org.apache.iceberg.spark.source.IcebergSource.catalogAndIdentifier(IcebergSource.java:141)
   	at org.apache.iceberg.spark.source.IcebergSource.extractIdentifier(IcebergSource.java:167)
   	at org.apache.spark.sql.DataFrameReader.$anonfun$load$1(DataFrameReader.scala:288)
    i think the problem is 
   `        Dataset<Row> posDeletes =
                   spark
                           .read()
                           .format("iceberg")
                           .option(SparkReadOptions.FILE_SCAN_TASK_SET_ID, groupId)
                           .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
                           .option(SparkReadOptions.FILE_OPEN_COST, "0")
                           .load(groupId);`
   it seems like spark 3.1 can not read table based on groupId?


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java:
##########
@@ -64,4 +64,10 @@ default DeleteReachableFiles deleteReachableFiles(String metadataLocation) {
     throw new UnsupportedOperationException(
         this.getClass().getName() + " does not implement deleteReachableFiles");
   }
+
+  /** Instantiates an action to rewrite position delete files all the files */

Review Comment:
   minor: Is there a typo in the Javadoc?



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,
+      List<PositionDeletesScanTask> fileScanTasks) {
+    this.info = info;
+    this.positionDeletesScanTasks = fileScanTasks;
+  }
+
+  public RewritePositionDeleteFiles.PositionDeleteGroupInfo info() {
+    return info;
+  }
+
+  public List<PositionDeletesScanTask> scans() {

Review Comment:
   optional: What about `tasks`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java:
##########
@@ -101,6 +101,8 @@
 
   public static final long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 100L * 1024 * 1024 * 1024; // 100 GB
 
+  protected static final long SPLIT_OVERHEAD = 5 * 1024;

Review Comment:
   Shall we also drop this constant from `SparkBinPackDataRewriter`? It is no longer used there.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark
+            .read()
+            .format("iceberg")
+            .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId)
+            .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
+            .option(SparkReadOptions.FILE_OPEN_COST, "0")
+            .load(groupId);
+
+    // keep only valid position deletes
+    Dataset<Row> dataFiles = dataFiles(partitionType, partition);
+    Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path"));
+    Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi");
+
+    // write the packed deletes into new files where each split becomes a new file
+    joined
+        .sortWithinPartitions("file_path", "pos")
+        .write()
+        .format("iceberg")
+        .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId)
+        .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+        .mode("append")
+        .save("default.test_table" + ".position_deletes");

Review Comment:
   Hm, how does it work?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group

Review Comment:
   optional: `All` -> `all`?



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = table.spec().partitionType();
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+      StructLike emptyStruct = GenericRecord.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            // If a task uses an incompatible partition spec the data inside could contain values
+            // which
+            // belong to multiple partitions in the current spec. Treating all such files as
+            // un-partitioned and
+            // grouping them together helps to minimize new files made.
+            StructLike taskPartition =

Review Comment:
   I don't think this would apply in this case as the in and out spec will always be the same.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewritePositionDeleteFile Actions from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups fileSets to commit

Review Comment:
   minor: Why `fileSets` as a single word?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = table.spec().partitionType();

Review Comment:
   Yea.. i reworked this code to use the coereced table's partitionType, and added testPartitionEvolutionRemove() in the test class to verify.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = table.spec().partitionType();
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+      StructLike emptyStruct = GenericRecord.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            // If a task uses an incompatible partition spec the data inside could contain values
+            // which
+            // belong to multiple partitions in the current spec. Treating all such files as
+            // un-partitioned and
+            // grouping them together helps to minimize new files made.
+            StructLike taskPartition =

Review Comment:
   Good point, removed



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewritePositionDeleteFile Actions from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups fileSets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    RewriteFiles rewrite = table.newRewrite().validateFromSnapshot(startingSnapshotId);
+    rewrite.rewriteDeleteFiles(rewrittenDeleteFiles, addedDeleteFiles);
+
+    rewrite.commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should not
+   * throw any exceptions
+   *
+   * @param fileGroup group of files which has already been rewritten
+   */
+  public void abortFileGroup(RewritePositionDeleteGroup fileGroup) {
+    Preconditions.checkState(
+        fileGroup.addedDeleteFiles() != null, "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedDeleteFiles())

Review Comment:
   Exposed the method as public in CatalogUtil and now I use it



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

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

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


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


[GitHub] [iceberg] aokolnychyi merged pull request #7389: Spark 3.4: Implement rewrite position deletes

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


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark
+            .read()
+            .format("iceberg")
+            .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId)
+            .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
+            .option(SparkReadOptions.FILE_OPEN_COST, "0")
+            .load(groupId);
+
+    // keep only valid position deletes
+    Dataset<Row> dataFiles = dataFiles(partitionType, partition);
+    Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path"));
+    Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi");
+
+    // write the packed deletes into new files where each split becomes a new file
+    joined
+        .sortWithinPartitions("file_path", "pos")
+        .write()
+        .format("iceberg")
+        .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId)
+        .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+        .mode("append")
+        .save("default.test_table" + ".position_deletes");

Review Comment:
   Whoops, sorry!  was doing some test change and failed to revert, thanks for catching



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());

Review Comment:
   I made the change, but now the BaseSparkAction does not use the spark session with disabled AQE.  I guess that should be ok?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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

   Hi @ajantha-bhat , thanks for asking, i havent started yet but I had planned to work on it probably late this week or next week, after some cleanup of this patch.  Will you have something in next few days on that?


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -32,13 +32,19 @@ public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /** @deprecated will be removed in 1.4.0; use {@link #fetchNewFiles(Table, String)} instead. */
+  /**
+   * @deprecated will be removed in 1.4.0; use {@link
+   *     BaseFileRewriteCoordinator#fetchNewFiles(Table, String)} instead.

Review Comment:
   Fix javadoc link warnings



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -56,7 +56,7 @@
 
 /**
  * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
- * {@link PositionDeleteBatchWrite}.
+ * PositionDeleteBatchWrite.

Review Comment:
   Fixing javadoc link warnings (seems it cannot link to inner class javaDocs)



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java:
##########
@@ -32,13 +32,19 @@ public static FileRewriteCoordinator get() {
     return INSTANCE;
   }
 
-  /** @deprecated will be removed in 1.4.0; use {@link #fetchNewFiles(Table, String)} instead. */
+  /**
+   * @deprecated will be removed in 1.4.0; use {@link
+   *     BaseFileRewriteCoordinator#fetchNewFiles(Table, String)} instead.

Review Comment:
   Fixes javadoc link warnings



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction

Review Comment:
   minor: There is no `RewriteAction`.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark
+            .read()
+            .format("iceberg")
+            .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId)
+            .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
+            .option(SparkReadOptions.FILE_OPEN_COST, "0")
+            .load(groupId);
+
+    // keep only valid position deletes
+    Dataset<Row> dataFiles = dataFiles(partitionType, partition);
+    Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path"));
+    Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi");
+
+    // write the packed deletes into new files where each split becomes a new file
+    joined
+        .sortWithinPartitions("file_path", "pos")
+        .write()
+        .format("iceberg")
+        .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId)
+        .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+        .mode("append")
+        .save("default.test_table" + ".position_deletes");
+  }
+
+  /** Returns entries of {@link DataFilesTable} of specified partition */
+  private Dataset<Row> dataFiles(Types.StructType partitionType, StructLike partition) {
+    List<Types.NestedField> fields = partitionType.fields();
+    Optional<Column> condition =
+        IntStream.range(0, fields.size())
+            .mapToObj(
+                i -> {
+                  Class<?> type = fields.get(i).type().typeId().javaClass();
+                  Object value = partition.get(i, type);
+                  Column col = col("partition." + fields.get(i).name());
+                  return col.equalTo(value);
+                })
+            .reduce(Column::and);
+    if (condition.isPresent()) {
+      return SparkTableUtil.loadMetadataTable(spark, table(), MetadataTableType.DATA_FILES)

Review Comment:
   Could you manually check the partition pruning happens here?



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark

Review Comment:
   Are we making sure AQE is disabled? I believe the action calling this rewriter should clone the session and disable AQE but it may make sense to add an extra precondition here as well.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkPositionDeletesRewriter.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.actions;
+
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+class SparkPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    this.spark = spark;
+  }
+
+  @Override
+  public String description() {
+    return "POSITION-DELETES";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table(), MetadataTableType.POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // All position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =
+        spark
+            .read()
+            .format("iceberg")
+            .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId)
+            .option(SparkReadOptions.SPLIT_SIZE, splitSize(inputSize(group)))
+            .option(SparkReadOptions.FILE_OPEN_COST, "0")
+            .load(groupId);
+
+    // keep only valid position deletes
+    Dataset<Row> dataFiles = dataFiles(partitionType, partition);
+    Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path"));
+    Dataset<Row> joined = posDeletes.join(dataFiles, joinCond, "leftsemi");
+
+    // write the packed deletes into new files where each split becomes a new file
+    joined
+        .sortWithinPartitions("file_path", "pos")
+        .write()
+        .format("iceberg")
+        .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId)
+        .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, writeMaxFileSize())
+        .mode("append")
+        .save("default.test_table" + ".position_deletes");
+  }
+
+  /** Returns entries of {@link DataFilesTable} of specified partition */
+  private Dataset<Row> dataFiles(Types.StructType partitionType, StructLike partition) {
+    List<Types.NestedField> fields = partitionType.fields();
+    Optional<Column> condition =
+        IntStream.range(0, fields.size())
+            .mapToObj(
+                i -> {
+                  Class<?> type = fields.get(i).type().typeId().javaClass();
+                  Object value = partition.get(i, type);
+                  Column col = col("partition." + fields.get(i).name());
+                  return col.equalTo(value);
+                })
+            .reduce(Column::and);
+    if (condition.isPresent()) {
+      return SparkTableUtil.loadMetadataTable(spark, table(), MetadataTableType.DATA_FILES)

Review Comment:
   question: Do we have to scan the same version of the table? It is OK if someone modifies it in the meantime? Seems like it is not required but we will need to think a bit 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] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";

Review Comment:
   I was trying to make it same as RewriteDataFiles which also defines these constants again with a larger javadoc.  Can remove this (though it will make the class javadoc no longer show all the valid arguments)



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.

Review Comment:
   It should be fine to have an empty set of new delete files, right? If this happens, we simply drop old deletes.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";

Review Comment:
   I believe this is now part of `SizeBasedFileRewriter` that is used by the action?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Since it is a nested class, what about simply `FileGroupRewriteResult` and `FileGroupInfo`? We usually rarely add any context in the name for nested classes. Not a strong opinion, though. Would love to hear more from others.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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

   Failure is more related to https://github.com/apache/iceberg/pull/7422


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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a RewriteAction
+ * and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {
+  private final RewritePositionDeleteFiles.PositionDeleteGroupInfo info;
+  private final List<PositionDeletesScanTask> positionDeletesScanTasks;
+
+  private Set<DeleteFile> addedDeleteFiles = Collections.emptySet();
+
+  public RewritePositionDeleteGroup(
+      RewritePositionDeleteFiles.PositionDeleteGroupInfo info,

Review Comment:
   minor: Let's do a direct import to have shorter lines.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();

Review Comment:
   Shall we add default implementation to aggregate the results from individual results?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java:
##########
@@ -74,6 +74,12 @@ public RewriteFiles rewriteFiles(
     return rewriteFiles(filesToDelete, ImmutableSet.of(), filesToAdd, ImmutableSet.of());
   }
 
+  @Override
+  public RewriteFiles rewriteDeleteFiles(

Review Comment:
   Let's not add this too for now.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewritePositionDeleteFile Actions from different platforms to handle

Review Comment:
   minor: Can we use a Javadoc link? Also, `Actions` -> `actions`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java:
##########
@@ -56,7 +56,7 @@
 
 /**
  * {@link Write} class for rewriting position delete files from Spark. Responsible for creating
- * {@link PositionDeleteBatchWrite}.
+ * PositionDeleteBatchWrite.

Review Comment:
   What if you include the parent class?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();

Review Comment:
   Hm is this circular?  Added it to the RewritePositionDeletesSparkAction, hope that works.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,458 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY = ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY;
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike taskPartition = task.file().partition();
+        StructLike coerced =
+            PartitionUtil.coercePartition(partitionType, task.spec(), taskPartition);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition

Review Comment:
   Yea, comment was from old code, removed from here and another place.



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {

Review Comment:
   Changed.



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -18,16 +18,68 @@
  */
 package org.apache.iceberg.actions;
 
+import java.util.List;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.expressions.Expression;
+import org.immutables.value.Value;
 
 /**
  * An action for rewriting position delete files.
  *
  * <p>Generally used for optimizing the size and layout of position delete files within a table.
  */
+@Value.Enclosing
 public interface RewritePositionDeleteFiles

Review Comment:
   Deprecated



##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteGroup.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.actions;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupInfo;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Container class representing a set of position delete files to be rewritten by a {@link
+ * RewritePositionDeleteFiles} and the new files which have been written by the action.
+ */
+public class RewritePositionDeleteGroup {

Review Comment:
   Good point, 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] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          Maps.newHashMapWithExpectedSize(filesByPartition.size());
+
+      filesByPartition.forEach(
+          (partition, partitionScans) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionScans.scanTasks());
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              // use coerced partition for map key uniqueness, but return original partition
+              fileGroupsByPartition.put(partitionScans.originalPartition(), groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeleteGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeleteGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite Position Deletes Ready to be Committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeleteGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeleteGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups)
+          .suppressFailureWhenFinished()
+          .run(commitManager::abortFileGroup);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableResult.builder()
+        .rewriteResults(rewriteResults)
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeleteGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    RewritePositionDeletesCommitManager.CommitService commitService =
+        commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeleteGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<PositionDeleteGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeleteGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableResult.builder().rewriteResults(rewriteResults).build();
+  }
+
+  Stream<RewritePositionDeleteGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeleteGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            // as coerced partition used for map, return original partition
+                            PositionDeleteGroupInfo info =
+                                ImmutablePositionDeleteGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeleteGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(rewriteGroupComparator());
+  }
+
+  private Comparator<RewritePositionDeleteGroup> rewriteGroupComparator() {

Review Comment:
   Added a static method there, if that is what you meant?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder().build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<PartitionScans> filesByPartition = StructLikeMap.create(partitionType);
+
+      scanTasks.forEach(
+          task -> {
+            StructLike taskPartition = task.file().partition();
+            StructLike coerced =
+                PartitionUtil.coercePartition(partitionType, task.spec(), task.partition());
+
+            PartitionScans partitionScans = filesByPartition.get(coerced);
+            if (partitionScans == null) {
+              partitionScans = new PartitionScans(taskPartition);
+            }
+            partitionScans.add(task);
+            filesByPartition.put(coerced, partitionScans);
+          });
+
+      Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition =

Review Comment:
   As discussed offline, now pass the coerced partition tuple all the way to the result.  It is actually even more usable for user, as before we did not have specid for them to interpret the historical partitions.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java:
##########
@@ -101,6 +101,8 @@
 
   public static final long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 100L * 1024 * 1024 * 1024; // 100 GB
 
+  protected static final long SPLIT_OVERHEAD = 5 * 1024;

Review Comment:
   Shall we also drop this constant from `SparkBinPackDataRewriter`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =

Review Comment:
   for future: Can we explore the idea of having 2 helper methods: one for computing files by partition and another file groups by partition. Not in this PR.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction

Review Comment:
   Shouldn't this be called `RewritePositionDeleteFilesSparkAction`? This is public facing and we usually name it as the interface name + `SparkAction`.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              fileGroupsByPartition.put(partition, groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite position deletes ready to be committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeletesGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeletesGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups).suppressFailureWhenFinished().run(commitManager::abort);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<FileGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeletesGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableRewritePositionDeleteFiles.Result.builder()
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service

Review Comment:
   minor: Inconsistent usage of capital letters across 3 comments in this method.
   
   ```
   // Start Commit Service
   // Start rewrite tasks
   // Stop Commit service
   ```



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              fileGroupsByPartition.put(partition, groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite position deletes ready to be committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeletesGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeletesGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups).suppressFailureWhenFinished().run(commitManager::abort);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<FileGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeletesGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableRewritePositionDeleteFiles.Result.builder()
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    CommitService commitService = commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeletesGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<FileGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeletesGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableRewritePositionDeleteFiles.Result.builder()
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  Stream<RewritePositionDeletesGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeletesGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(

Review Comment:
   for future: Can we try refactoring this using some helper methods cause Spotless formats this in a weird way. Not in this PR.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackPositionDeletesRewriter.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import static org.apache.iceberg.MetadataTableType.POSITION_DELETES;
+import static org.apache.spark.sql.functions.col;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.IntStream;
+import org.apache.iceberg.DataFilesTable;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.SizeBasedPositionDeletesRewriter;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator;
+import org.apache.iceberg.spark.ScanTaskSetManager;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkTableCache;
+import org.apache.iceberg.spark.SparkTableUtil;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+
+class SparkBinPackPositionDeletesRewriter extends SizeBasedPositionDeletesRewriter {
+
+  private final SparkSession spark;
+  private final SparkTableCache tableCache = SparkTableCache.get();
+  private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get();
+  private final PositionDeletesRewriteCoordinator coordinator =
+      PositionDeletesRewriteCoordinator.get();
+
+  SparkBinPackPositionDeletesRewriter(SparkSession spark, Table table) {
+    super(table);
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    this.spark = spark.cloneSession();
+    this.spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+  }
+
+  @Override
+  public String description() {
+    return "BIN-PACK";
+  }
+
+  @Override
+  public Set<DeleteFile> rewrite(List<PositionDeletesScanTask> group) {
+    String groupId = UUID.randomUUID().toString();
+    Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table(), POSITION_DELETES);
+    try {
+      tableCache.add(groupId, deletesTable);
+      taskSetManager.stageTasks(deletesTable, groupId, group);
+
+      doRewrite(groupId, group);
+
+      return coordinator.fetchNewFiles(deletesTable, groupId);
+    } finally {
+      tableCache.remove(groupId);
+      taskSetManager.removeTasks(deletesTable, groupId);
+      coordinator.clearRewrite(deletesTable, groupId);
+    }
+  }
+
+  protected void doRewrite(String groupId, List<PositionDeletesScanTask> group) {
+    // all position deletes are of the same partition, because they are in same file group
+    Preconditions.checkArgument(group.size() > 0, "Empty group");
+    Types.StructType partitionType = group.get(0).spec().partitionType();
+    StructLike partition = group.get(0).partition();
+
+    // read the deletes packing them into splits of the required size
+    Dataset<Row> posDeletes =

Review Comment:
   minor: We frequently add `DF` suffix for variables referring to `Dataset<Row>`.
   
   ```
   posDeleteDF
   dataFileDF
   validPosDeleteDF
   ```



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              fileGroupsByPartition.put(partition, groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesGroup rewriteDeleteFiles(

Review Comment:
   Just making sure that this is indeed used for testing.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              fileGroupsByPartition.put(partition, groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite position deletes ready to be committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {
+    return new RewritePositionDeletesCommitManager(table);
+  }
+
+  private Result doExecute(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    ConcurrentLinkedQueue<RewritePositionDeletesGroup> rewrittenGroups =
+        Queues.newConcurrentLinkedQueue();
+
+    Tasks.Builder<RewritePositionDeletesGroup> rewriteTaskBuilder =
+        Tasks.foreach(groupStream)
+            .executeWith(rewriteService)
+            .stopOnFailure()
+            .noRetry()
+            .onFailure(
+                (fileGroup, exception) ->
+                    LOG.warn(
+                        "Failure during rewrite process for group {}",
+                        fileGroup.info(),
+                        exception));
+
+    try {
+      rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(ctx, fileGroup)));
+    } catch (Exception e) {
+      // At least one rewrite group failed, clean up all completed rewrites
+      LOG.error(
+          "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to "
+              + "be rewritten. This error occurred during the writing of new files, not during the commit process. This "
+              + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling "
+              + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished "
+              + "being written.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_ENABLED,
+          rewrittenGroups.size(),
+          e);
+
+      Tasks.foreach(rewrittenGroups).suppressFailureWhenFinished().run(commitManager::abort);
+      throw e;
+    } finally {
+      rewriteService.shutdown();
+    }
+
+    try {
+      commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups));
+    } catch (ValidationException | CommitFailedException e) {
+      String errorMessage =
+          String.format(
+              "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that "
+                  + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of "
+                  + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. "
+                  + "Separate smaller rewrite commits can succeed independently while any commits that conflict with "
+                  + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table "
+                  + "history, one for each commit.",
+              PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS);
+      throw new RuntimeException(errorMessage, e);
+    }
+
+    List<FileGroupRewriteResult> rewriteResults =
+        rewrittenGroups.stream()
+            .map(RewritePositionDeletesGroup::asResult)
+            .collect(Collectors.toList());
+
+    return ImmutableRewritePositionDeleteFiles.Result.builder()
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  private Result doExecuteWithPartialProgress(
+      RewriteExecutionContext ctx,
+      Stream<RewritePositionDeletesGroup> groupStream,
+      RewritePositionDeletesCommitManager commitManager) {
+    ExecutorService rewriteService = rewriteService();
+
+    // Start Commit Service
+    int groupsPerCommit = IntMath.divide(ctx.totalGroupCount(), maxCommits, RoundingMode.CEILING);
+    CommitService commitService = commitManager.service(groupsPerCommit);
+    commitService.start();
+
+    // Start rewrite tasks
+    Tasks.foreach(groupStream)
+        .suppressFailureWhenFinished()
+        .executeWith(rewriteService)
+        .noRetry()
+        .onFailure(
+            (fileGroup, exception) ->
+                LOG.error("Failure during rewrite group {}", fileGroup.info(), exception))
+        .run(fileGroup -> commitService.offer(rewriteDeleteFiles(ctx, fileGroup)));
+    rewriteService.shutdown();
+
+    // Stop Commit service
+    commitService.close();
+    List<RewritePositionDeletesGroup> commitResults = commitService.results();
+    if (commitResults.size() == 0) {
+      LOG.error(
+          "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual "
+              + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation "
+              + "into smaller commits.",
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS);
+    }
+
+    List<FileGroupRewriteResult> rewriteResults =
+        commitResults.stream()
+            .map(RewritePositionDeletesGroup::asResult)
+            .collect(Collectors.toList());
+    return ImmutableRewritePositionDeleteFiles.Result.builder()
+        .rewriteResults(rewriteResults)
+        .build();
+  }
+
+  Stream<RewritePositionDeletesGroup> toGroupStream(
+      RewriteExecutionContext ctx,
+      Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+    Stream<RewritePositionDeletesGroup> rewriteFileGroupStream =
+        groupsByPartition.entrySet().stream()
+            .flatMap(
+                e -> {
+                  StructLike partition = e.getKey();
+                  List<List<PositionDeletesScanTask>> scanGroups = e.getValue();
+                  return scanGroups.stream()
+                      .map(
+                          tasks -> {
+                            int globalIndex = ctx.currentGlobalIndex();
+                            int partitionIndex = ctx.currentPartitionIndex(partition);
+                            FileGroupInfo info =
+                                ImmutableRewritePositionDeleteFiles.FileGroupInfo.builder()
+                                    .globalIndex(globalIndex)
+                                    .partitionIndex(partitionIndex)
+                                    .partition(partition)
+                                    .build();
+                            return new RewritePositionDeletesGroup(info, tasks);
+                          });
+                });
+
+    return rewriteFileGroupStream.sorted(RewritePositionDeletesGroup.comparator(rewriteJobOrder));
+  }
+
+  private void validateAndInitOptions() {
+    Set<String> validOptions = Sets.newHashSet(rewriter.validOptions());
+    validOptions.addAll(VALID_OPTIONS);
+
+    Set<String> invalidKeys = Sets.newHashSet(options().keySet());
+    invalidKeys.removeAll(validOptions);
+
+    Preconditions.checkArgument(
+        invalidKeys.isEmpty(),
+        "Cannot use options %s, they are not supported by the action or the rewriter %s",
+        invalidKeys,
+        rewriter.description());
+
+    rewriter.init(options());
+
+    this.maxConcurrentFileGroupRewrites =
+        PropertyUtil.propertyAsInt(
+            options(),
+            MAX_CONCURRENT_FILE_GROUP_REWRITES,
+            MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT);
+
+    this.maxCommits =
+        PropertyUtil.propertyAsInt(
+            options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT);
+
+    this.partialProgressEnabled =
+        PropertyUtil.propertyAsBoolean(
+            options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT);
+
+    this.rewriteJobOrder =
+        RewriteJobOrder.fromName(
+            PropertyUtil.propertyAsString(options(), REWRITE_JOB_ORDER, REWRITE_JOB_ORDER_DEFAULT));
+
+    Preconditions.checkArgument(
+        maxConcurrentFileGroupRewrites >= 1,
+        "Cannot set %s to %s, the value must be positive.",
+        MAX_CONCURRENT_FILE_GROUP_REWRITES,
+        maxConcurrentFileGroupRewrites);
+
+    Preconditions.checkArgument(
+        !partialProgressEnabled || maxCommits > 0,
+        "Cannot set %s to %s, the value must be positive when %s is true",
+        PARTIAL_PROGRESS_MAX_COMMITS,
+        maxCommits,
+        PARTIAL_PROGRESS_ENABLED);
+  }
+
+  private String jobDesc(RewritePositionDeletesGroup group, RewriteExecutionContext ctx) {
+    StructLike partition = group.info().partition();
+    if (partition.size() > 0) {
+      return String.format(
+          "Rewriting %d position delete files (%s, file group %d/%d, %s (%d/%d)) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          partition,
+          group.info().partitionIndex(),
+          ctx.groupsInPartition(partition),
+          table.name());
+    } else {
+      return String.format(
+          "Rewriting %d position files (%s, file group %d/%d) in %s",
+          group.rewrittenDeleteFiles().size(),
+          rewriter.description(),
+          group.info().globalIndex(),
+          ctx.totalGroupCount(),
+          table.name());
+    }
+  }
+
+  static class RewriteExecutionContext {
+    private final Map<StructLike, Integer> numGroupsByPartition;
+    private final int totalGroupCount;
+    private final Map<StructLike, Integer> partitionIndexMap;
+    private final AtomicInteger groupIndex;
+
+    RewriteExecutionContext(
+        Map<StructLike, List<List<PositionDeletesScanTask>>> groupsByPartition) {
+      this.numGroupsByPartition =

Review Comment:
   Hm, I think we should use `StructLikeMap` here too.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());

Review Comment:
   That should be fine as it does not depend on disabled AQE.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final SparkPositionDeletesBinPackRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesBinPackRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder().build();

Review Comment:
   That works.



##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteSparkAction.java:
##########
@@ -0,0 +1,456 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.Partitioning;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService;
+import org.apache.iceberg.actions.RewritePositionDeletesGroup;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PartitionUtil;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link RewritePositionDeleteFiles}. */
+public class RewritePositionDeleteSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeleteSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG = LoggerFactory.getLogger(RewritePositionDeleteSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          REWRITE_JOB_ORDER);
+  private static final Result EMPTY_RESULT =
+      ImmutableRewritePositionDeleteFiles.Result.builder().build();
+
+  private final Table table;
+  private final SparkBinPackPositionDeletesRewriter rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeleteSparkAction(SparkSession spark, Table table) {
+    super(spark);
+    this.table = table;
+    this.rewriter = new SparkBinPackPositionDeletesRewriter(spark(), table);
+  }
+
+  @Override
+  protected RewritePositionDeleteSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeleteSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public RewritePositionDeleteFiles.Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return EMPTY_RESULT;
+    }
+
+    Stream<RewritePositionDeletesGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  private Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);
+    CloseableIterable<PositionDeletesScanTask> scanTasks =
+        CloseableIterable.transform(
+            deletesTable.newBatchScan().ignoreResiduals().planFiles(),
+            t -> (PositionDeletesScanTask) t);
+
+    try {
+      StructType partitionType = Partitioning.partitionType(table);
+      StructLikeMap<List<PositionDeletesScanTask>> filesByPartition =
+          StructLikeMap.create(partitionType);
+
+      for (PositionDeletesScanTask task : scanTasks) {
+        StructLike coerced = coercePartition(task, partitionType);
+
+        List<PositionDeletesScanTask> partitionTasks = filesByPartition.get(coerced);
+        if (partitionTasks == null) {
+          partitionTasks = Lists.newArrayList();
+        }
+        partitionTasks.add(task);
+        filesByPartition.put(coerced, partitionTasks);
+      }
+
+      StructLikeMap<List<List<PositionDeletesScanTask>>> fileGroupsByPartition =
+          StructLikeMap.create(partitionType);
+
+      filesByPartition.forEach(
+          (partition, partitionTasks) -> {
+            Iterable<List<PositionDeletesScanTask>> plannedFileGroups =
+                rewriter.planFileGroups(partitionTasks);
+            List<List<PositionDeletesScanTask>> groups = ImmutableList.copyOf(plannedFileGroups);
+            if (groups.size() > 0) {
+              fileGroupsByPartition.put(partition, groups);
+            }
+          });
+
+      return fileGroupsByPartition;
+    } finally {
+      try {
+        scanTasks.close();
+      } catch (IOException io) {
+        LOG.error("Cannot properly close file iterable while planning for rewrite", io);
+      }
+    }
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesGroup rewriteDeleteFiles(
+      RewriteExecutionContext ctx, RewritePositionDeletesGroup fileGroup) {
+    String desc = jobDesc(fileGroup, ctx);
+    Set<DeleteFile> addedFiles =
+        withJobGroupInfo(
+            newJobGroupInfo("REWRITE-POSITION-DELETES", desc),
+            () -> rewriter.rewrite(fileGroup.tasks()));
+
+    fileGroup.setOutputFiles(addedFiles);
+    LOG.info("Rewrite position deletes ready to be committed - {}", desc);
+    return fileGroup;
+  }
+
+  private ExecutorService rewriteService() {
+    return MoreExecutors.getExitingExecutorService(
+        (ThreadPoolExecutor)
+            Executors.newFixedThreadPool(
+                maxConcurrentFileGroupRewrites,
+                new ThreadFactoryBuilder()
+                    .setNameFormat("Rewrite-Position-Delete-Service-%d")
+                    .build()));
+  }
+
+  @VisibleForTesting
+  RewritePositionDeletesCommitManager commitManager() {

Review Comment:
   Same here about visibility.



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java:
##########
@@ -178,6 +180,16 @@ protected long inputSize(List<T> group) {
     return group.stream().mapToLong(ContentScanTask::length).sum();
   }
 
+  /**
+   * Returns the smallest of our max write file threshold and our estimated split size based on the
+   * number of output files we want to generate. Add an overhead onto the estimated split size to
+   * try to avoid small errors in size creating brand-new files.
+   */
+  protected long splitSize(long inputSize) {

Review Comment:
   Moved from SparkBinPackDataRewriter to re-use



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/RewriteFiles.java:
##########
@@ -57,6 +57,16 @@ default RewriteFiles rewriteFiles(Set<DataFile> filesToDelete, Set<DataFile> fil
   RewriteFiles rewriteFiles(
       Set<DataFile> filesToDelete, Set<DataFile> filesToAdd, long sequenceNumber);
 
+  /**
+   * Add a rewrite that replaces one set of delete files with another set that contains the same
+   * data.
+   *
+   * @param filesToDelete files that will be replaced (deleted), cannot be null or empty.
+   * @param filesToAdd files that will be added, cannot be null or empty.
+   * @return this for method chaining
+   */
+  RewriteFiles rewriteDeleteFiles(Set<DeleteFile> filesToDelete, Set<DeleteFile> filesToAdd);

Review Comment:
   We should also reconsider this API. Too many overloaded methods. Let me think.



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

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

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


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.util.Set;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.RewriteFiles;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.exceptions.CommitStateUnknownException;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Functionality used by RewritePositionDeleteFile Actions from different platforms to handle
+ * commits.
+ */
+public class RewritePositionDeletesCommitManager {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesCommitManager.class);
+
+  private final Table table;
+  private final long startingSnapshotId;
+
+  public RewritePositionDeletesCommitManager(Table table) {
+    this.table = table;
+    this.startingSnapshotId = table.currentSnapshot().snapshotId();
+  }
+
+  /**
+   * Perform a commit operation on the table adding and removing files as required for this set of
+   * file groups
+   *
+   * @param fileGroups fileSets to commit
+   */
+  public void commitFileGroups(Set<RewritePositionDeleteGroup> fileGroups) {
+    Set<DeleteFile> rewrittenDeleteFiles = Sets.newHashSet();
+    Set<DeleteFile> addedDeleteFiles = Sets.newHashSet();
+    for (RewritePositionDeleteGroup group : fileGroups) {
+      rewrittenDeleteFiles.addAll(group.rewrittenDeleteFiles());
+      addedDeleteFiles.addAll(group.addedDeleteFiles());
+    }
+
+    RewriteFiles rewrite = table.newRewrite().validateFromSnapshot(startingSnapshotId);
+    rewrite.rewriteDeleteFiles(rewrittenDeleteFiles, addedDeleteFiles);
+
+    rewrite.commit();
+  }
+
+  /**
+   * Clean up a specified file set by removing any files created for that operation, should not
+   * throw any exceptions
+   *
+   * @param fileGroup group of files which has already been rewritten
+   */
+  public void abortFileGroup(RewritePositionDeleteGroup fileGroup) {
+    Preconditions.checkState(
+        fileGroup.addedDeleteFiles() != null, "Cannot abort a fileGroup that was not rewritten");
+
+    Tasks.foreach(fileGroup.addedDeleteFiles())

Review Comment:
   Shall we use bulk deletes if possible? Can we use `CatalogUtil$deleteFiles`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java:
##########
@@ -101,6 +101,8 @@
 
   public static final long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 100L * 1024 * 1024 * 1024; // 100 GB
 
+  protected static final long SPLIT_OVERHEAD = 5 * 1024;

Review Comment:
   minor: `protected` -> `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] aokolnychyi commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeletesSparkAction.java:
##########
@@ -0,0 +1,511 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.actions;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DeleteFile;
+import org.apache.iceberg.MetadataTableType;
+import org.apache.iceberg.MetadataTableUtils;
+import org.apache.iceberg.PositionDeletesScanTask;
+import org.apache.iceberg.RewriteJobOrder;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.FileRewriter;
+import org.apache.iceberg.actions.ImmutablePositionDeleteGroupInfo;
+import org.apache.iceberg.actions.ImmutableResult;
+import org.apache.iceberg.actions.RewritePositionDeleteFiles;
+import org.apache.iceberg.actions.RewritePositionDeleteGroup;
+import org.apache.iceberg.actions.RewritePositionDeletesCommitManager;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.relocated.com.google.common.math.IntMath;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors;
+import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.iceberg.types.Types.StructType;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.StructLikeMap;
+import org.apache.iceberg.util.Tasks;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.internal.SQLConf;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Spark implementation of {@link org.apache.iceberg.actions.RewritePositionDeleteFiles}. */
+public class RewritePositionDeletesSparkAction
+    extends BaseSnapshotUpdateSparkAction<RewritePositionDeletesSparkAction>
+    implements RewritePositionDeleteFiles {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(RewritePositionDeletesSparkAction.class);
+  private static final Set<String> VALID_OPTIONS =
+      ImmutableSet.of(
+          MAX_CONCURRENT_FILE_GROUP_REWRITES,
+          MAX_FILE_GROUP_SIZE_BYTES,
+          PARTIAL_PROGRESS_ENABLED,
+          PARTIAL_PROGRESS_MAX_COMMITS,
+          TARGET_FILE_SIZE_BYTES,
+          REWRITE_JOB_ORDER);
+
+  private final Table table;
+  private final FileRewriter<PositionDeletesScanTask, DeleteFile> rewriter;
+
+  private int maxConcurrentFileGroupRewrites;
+  private int maxCommits;
+  private boolean partialProgressEnabled;
+  private RewriteJobOrder rewriteJobOrder;
+
+  RewritePositionDeletesSparkAction(SparkSession spark, Table table) {
+    super(spark.cloneSession());
+
+    // Disable Adaptive Query Execution as this may change the output partitioning of our write
+    spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false);
+    this.table = table;
+    this.rewriter = new SparkPositionDeletesRewriter(spark, table);
+  }
+
+  @Override
+  protected RewritePositionDeletesSparkAction self() {
+    return this;
+  }
+
+  @Override
+  public RewritePositionDeletesSparkAction filter(Expression expression) {
+    throw new UnsupportedOperationException("Regular filters not supported yet.");
+  }
+
+  @Override
+  public Result execute() {
+    if (table.currentSnapshot() == null) {
+      LOG.info("Nothing found to rewrite in empty table {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    validateAndInitOptions();
+
+    Map<StructLike, List<List<PositionDeletesScanTask>>> fileGroupsByPartition = planFileGroups();
+    RewriteExecutionContext ctx = new RewriteExecutionContext(fileGroupsByPartition);
+
+    if (ctx.totalGroupCount() == 0) {
+      LOG.info("Nothing found to rewrite in {}", table.name());
+      return ImmutableResult.builder()
+          .rewrittenDeleteFilesCount(0)
+          .addedDeleteFilesCount(0)
+          .rewrittenBytesCount(0)
+          .addedBytesCount(0)
+          .build();
+    }
+
+    Stream<RewritePositionDeleteGroup> groupStream = toGroupStream(ctx, fileGroupsByPartition);
+
+    RewritePositionDeletesCommitManager commitManager = commitManager();
+    if (partialProgressEnabled) {
+      return doExecuteWithPartialProgress(ctx, groupStream, commitManager);
+    } else {
+      return doExecute(ctx, groupStream, commitManager);
+    }
+  }
+
+  Map<StructLike, List<List<PositionDeletesScanTask>>> planFileGroups() {
+    Table deletesTable =
+        MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES);

Review Comment:
   Is it equivalent to `loadMetadataTable()` in `BaseSparkAction`?



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Sorry , not familiar, if you can point me to that



##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -28,6 +32,80 @@
 public interface RewritePositionDeleteFiles
     extends SnapshotUpdate<RewritePositionDeleteFiles, RewritePositionDeleteFiles.Result> {
 
+  /**
+   * Enable committing groups of files (see max-file-group-size-bytes) prior to the entire rewrite
+   * completing. This will produce additional commits but allow for progress even if some groups
+   * fail to commit. This setting will not change the correctness of the rewrite operation as file
+   * groups can be compacted independently.
+   *
+   * <p>The default is false, which produces a single commit when the entire job has completed.
+   */
+  String PARTIAL_PROGRESS_ENABLED = "partial-progress.enabled";
+
+  boolean PARTIAL_PROGRESS_ENABLED_DEFAULT = false;
+
+  /**
+   * The maximum amount of Iceberg commits that this rewrite is allowed to produce if partial
+   * progress is enabled. This setting has no effect if partial progress is disabled.
+   */
+  String PARTIAL_PROGRESS_MAX_COMMITS = "partial-progress.max-commits";
+
+  int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10;
+
+  /**
+   * The entire rewrite operation is broken down into pieces based on partitioning and within
+   * partitions based on size into groups. These sub-units of the rewrite are referred to as file
+   * groups. The largest amount of data that should be compacted in a single group is controlled by
+   * {@link #MAX_FILE_GROUP_SIZE_BYTES}. This helps with breaking down the rewriting of very large
+   * partitions which may not be rewritable otherwise due to the resource constraints of the
+   * cluster. For example a sort based rewrite may not scale to terabyte sized partitions, those
+   * partitions need to be worked on in small subsections to avoid exhaustion of resources.
+   *
+   * <p>When grouping files, the underlying rewrite strategy will use this value as to limit the
+   * files which will be included in a single file group. A group will be processed by a single
+   * framework "action". For example, in Spark this means that each group would be rewritten in its
+   * own Spark action. A group will never contain files for multiple output partitions.
+   */
+  String MAX_FILE_GROUP_SIZE_BYTES = "max-file-group-size-bytes";

Review Comment:
   Removed 



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

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

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


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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();
 
     /** Returns the count of the added delete files. */
     int addedDeleteFilesCount();
+
+    /** Returns the number of bytes of position deletes that have been rewritten */
+    long rewrittenBytesCount();
+
+    /** Returns the number of bytes of newly added position deletes */
+    long addedBytesCount();
+  }
+
+  /**
+   * For a particular position delete file group, the number of position delete files which are
+   * newly created and the number of files which were formerly part of the table but have been
+   * rewritten.
+   */
+  @Value.Immutable
+  interface PositionDeleteGroupRewriteResult {

Review Comment:
   Oh I see, well I think the problem is not that, they're no enclosing each other.  it's just they all end up in the same place.  That being said, FileGroupRewriteResult seems to me a bit particular to RewriteDataFiles somehow



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

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

For queries about this service, please contact Infrastructure at:
users@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 #7389: Spark 3.4: Implement rewrite position deletes

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


##########
api/src/main/java/org/apache/iceberg/actions/RewritePositionDeleteFiles.java:
##########
@@ -41,11 +119,60 @@
   RewritePositionDeleteFiles filter(Expression expression);
 
   /** The action result that contains a summary of the execution. */
+  @Value.Immutable
   interface Result {
+    List<PositionDeleteGroupRewriteResult> rewriteResults();
+
     /** Returns the count of the position deletes that been rewritten. */
     int rewrittenDeleteFilesCount();

Review Comment:
   Good idea, 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